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

[01/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Repository: asterixdb
Updated Branches:
  refs/heads/master 028537d1f -> c3c235743


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index f269a11..72941f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -47,8 +47,22 @@ public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePush
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
             throws HyracksDataException {
+        this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
+                minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
+                missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null, null);
+    }
+
+    public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
+            int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                searchCallbackProceedResultTrueValue);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
         if (lowKeyFields != null && lowKeyFields.length > 0) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ILSMIndexCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ILSMIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ILSMIndexCursor.java
index 979ff51..67ac5a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ILSMIndexCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ILSMIndexCursor.java
@@ -33,4 +33,14 @@ public interface ILSMIndexCursor extends IIndexCursor {
      * @return the max tuple of the corresponding component's filter
      */
     ITupleReference getFilterMaxTuple();
+
+    /**
+     * Returns the result of the current SearchOperationCallback.proceed().
+     * This method is used for the secondary-index searches.
+     *
+     * @return true if SearchOperationCallback.proceed() succeeded
+     *         false otherwise
+     */
+    boolean getSearchOperationCallbackProceedResult();
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index 41fdc41..f562379 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -90,11 +90,27 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
     protected boolean failed = false;
     private final IOperatorStats stats;
 
+    // Used when the result of the search operation callback needs to be passed.
+    protected boolean appendSearchCallbackProceedResult;
+    protected byte[] searchCallbackProceedResultFalseValue;
+    protected byte[] searchCallbackProceedResultTrueValue;
+
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
             throws HyracksDataException {
+        this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
+                null);
+    }
+
+    public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
         this.ctx = ctx;
         this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
         this.retainInput = retainInput;
@@ -115,6 +131,9 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
             maxFilterKey = new PermutingFrameTupleReference();
             maxFilterKey.setFieldPermutation(maxFilterFieldIndexes);
         }
+        this.appendSearchCallbackProceedResult = appendSearchCallbackProceedResult;
+        this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
+        this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
         stats = new OperatorStats(getDisplayName());
         if (ctx.getStatsCollector() != null) {
             ctx.getStatsCollector().add(stats);
@@ -139,8 +158,15 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
         accessor = new FrameTupleAccessor(inputRecDesc);
         if (retainMissing) {
             int fieldCount = getFieldCount();
-            nonMatchTupleBuild = new ArrayTupleBuilder(fieldCount);
+            // Field count in case searchCallback.proceed() result is needed.
+            int finalFieldCount = appendSearchCallbackProceedResult ? fieldCount + 1 : fieldCount;
+            nonMatchTupleBuild = new ArrayTupleBuilder(finalFieldCount);
             buildMissingTuple(fieldCount, nonMatchTupleBuild, nonMatchWriter);
+            if (appendSearchCallbackProceedResult) {
+                // Writes the success result in the last field in case we need to write down
+                // the result of searchOperationCallback.proceed(). This value can't be missing even for this case.
+                writeSearchCallbackProceedResult(nonMatchTupleBuild, true);
+            }
         } else {
             nonMatchTupleBuild = null;
         }
@@ -183,6 +209,10 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
             }
             ITupleReference tuple = cursor.getTuple();
             writeTupleToOutput(tuple);
+            if (appendSearchCallbackProceedResult) {
+                writeSearchCallbackProceedResult(tb,
+                        ((ILSMIndexCursor) cursor).getSearchOperationCallbackProceedResult());
+            }
             if (appendIndexFilter) {
                 writeFilterTupleToOutput(((ILSMIndexCursor) cursor).getFilterMinTuple());
                 writeFilterTupleToOutput(((ILSMIndexCursor) cursor).getFilterMaxTuple());
@@ -274,6 +304,18 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
         }
     }
 
+    /**
+     * Write the result of a SearchCallback.proceed() if it is needed.
+     */
+    private void writeSearchCallbackProceedResult(ArrayTupleBuilder atb, boolean searchCallbackProceedResult)
+            throws HyracksDataException {
+        if (!searchCallbackProceedResult) {
+            atb.addField(searchCallbackProceedResultFalseValue, 0, searchCallbackProceedResultFalseValue.length);
+        } else {
+            atb.addField(searchCallbackProceedResultTrueValue, 0, searchCallbackProceedResultTrueValue.length);
+        }
+    }
+
     private void writeFilterTupleToOutput(ITupleReference tuple) throws IOException {
         if (tuple != null) {
             writeTupleToOutput(tuple);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
----------------------------------------------------------------------
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 78564fd..1209e17 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
@@ -58,6 +58,7 @@ public class LSMBTreePointSearchCursor extends EnforcedIndexCursor implements IL
     private int foundIn = -1;
     private ITupleReference frameTuple;
     private List<ILSMComponent> operationalComponents;
+    private boolean resultOfSearchCallbackProceed = false;
 
     private final long[] hashes = BloomFilter.createHashArray();
 
@@ -82,7 +83,10 @@ public class LSMBTreePointSearchCursor extends EnforcedIndexCursor implements IL
                 btreeCursors[i].next();
                 // We use the predicate's to lock the key instead of the tuple that we get from cursor
                 // to avoid copying the tuple when we do the "unlatch dance".
-                if (reconciled || searchCallback.proceed(predicate.getLowKey())) {
+                if (!reconciled) {
+                    resultOfSearchCallbackProceed = searchCallback.proceed(predicate.getLowKey());
+                }
+                if (reconciled || resultOfSearchCallbackProceed) {
                     // if proceed is successful, then there's no need for doing the "unlatch dance"
                     if (((ILSMTreeTupleReference) btreeCursors[i].getTuple()).isAntimatter()) {
                         if (reconciled) {
@@ -238,4 +242,9 @@ public class LSMBTreePointSearchCursor extends EnforcedIndexCursor implements IL
             }
         }
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return resultOfSearchCallbackProceed;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 5d23fef..d8feab1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -52,6 +52,7 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
     private BTreeAccessor[] btreeAccessors;
     private ArrayTupleBuilder tupleBuilder;
     private boolean canCallProceed = true;
+    private boolean resultOfSearchCallbackProceed = false;
     private int tupleFromMemoryComponentCount = 0;
 
     public LSMBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx) {
@@ -104,46 +105,49 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
         while (!outputPriorityQueue.isEmpty() || needPushElementIntoQueue) {
             if (!outputPriorityQueue.isEmpty()) {
                 PriorityQueueElement queueHead = outputPriorityQueue.peek();
-                if (canCallProceed && includeMutableComponent && !searchCallback.proceed(queueHead.getTuple())) {
-                    // In case proceed() fails and there is an in-memory component,
-                    // we can't simply use this element since there might be a change.
-                    PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
-                    if (mutableElement != null) {
-                        // Copies the current queue head
-                        if (tupleBuilder == null) {
-                            tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                if (canCallProceed && includeMutableComponent) {
+                    resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
+                    if (!resultOfSearchCallbackProceed) {
+                        // In case proceed() fails and there is an in-memory component,
+                        // we can't simply use this element since there might be a change.
+                        PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
+                        if (mutableElement != null) {
+                            // Copies the current queue head
+                            if (tupleBuilder == null) {
+                                tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                            }
+                            TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
+                            copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+                            // Unlatches/unpins the leaf page of the index.
+                            rangeCursors[0].close();
+                            // Reconcile.
+                            searchCallback.reconcile(copyTuple);
+                            // Re-traverses the index.
+                            reusablePred.setLowKey(copyTuple, true);
+                            btreeAccessors[0].search(rangeCursors[0], reusablePred);
+                            pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
+                            // now that we have completed the search and we have latches over the pages,
+                            // it is safe to complete the operation.. but as per the API of the callback
+                            // we only complete if we're producing this tuple
+                            // get head again
+                            queueHead = outputPriorityQueue.peek();
+                            /*
+                             * We need to restart in one of two cases:
+                             * 1. no more elements in the priority queue.
+                             * 2. the key of the head has changed (which means we need to call proceed)
+                             */
+                            if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
+                                // cancel since we're not continuing
+                                searchCallback.cancel(copyTuple);
+                                continue;
+                            }
+                            searchCallback.complete(copyTuple);
+                            // it is safe to proceed now
+                        } else {
+                            // There are no more elements in the memory component.. can safely skip locking for the
+                            // remaining operations
+                            includeMutableComponent = false;
                         }
-                        TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
-                        copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-                        // Unlatches/unpins the leaf page of the index.
-                        rangeCursors[0].close();
-                        // Reconcile.
-                        searchCallback.reconcile(copyTuple);
-                        // Re-traverses the index.
-                        reusablePred.setLowKey(copyTuple, true);
-                        btreeAccessors[0].search(rangeCursors[0], reusablePred);
-                        pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
-                        // now that we have completed the search and we have latches over the pages,
-                        // it is safe to complete the operation.. but as per the API of the callback
-                        // we only complete if we're producing this tuple
-                        // get head again
-                        queueHead = outputPriorityQueue.peek();
-                        /*
-                         * We need to restart in one of two cases:
-                         * 1. no more elements in the priority queue.
-                         * 2. the key of the head has changed (which means we need to call proceed)
-                         */
-                        if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
-                            // cancel since we're not continuing
-                            searchCallback.cancel(copyTuple);
-                            continue;
-                        }
-                        searchCallback.complete(copyTuple);
-                        // it is safe to proceed now
-                    } else {
-                        // There are no more elements in the memory component.. can safely skip locking for the
-                        // remaining operations
-                        includeMutableComponent = false;
                     }
                 }
 
@@ -368,4 +372,10 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
         initPriorityQueue();
         canCallProceed = true;
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return resultOfSearchCallbackProceed;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index 02574ca..baf0d4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -104,4 +104,9 @@ public class LSMBTreeSearchCursor extends EnforcedIndexCursor implements ILSMInd
     public ITupleReference getFilterMaxTuple() {
         return currentCursor.getFilterMaxTuple();
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
index 8dcbcc4..08e7b91 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
@@ -166,4 +166,10 @@ public abstract class LSMBTreeWithBuddyAbstractCursor extends EnforcedIndexCurso
     public ITupleReference doGetTuple() {
         return frameTuple;
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 900ee32..445a005 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -311,4 +311,9 @@ public abstract class LSMIndexSearchCursor extends EnforcedIndexCursor implement
             throws HyracksDataException {
         return cmp.compare(tupleA, tupleB);
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index fea9373..4d444b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -216,4 +216,10 @@ public class LSMInvertedIndexSearchCursor extends EnforcedIndexCursor implements
         }
         return operationalComponents.get(accessorIndex).getLSMComponentFilter();
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index d41e406..176f767 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
 import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 
@@ -54,7 +55,7 @@ public abstract class LSMRTreeAbstractCursor extends EnforcedIndexCursor impleme
     protected RTreeAccessor[] rtreeAccessors;
     protected BTreeAccessor[] btreeAccessors;
     protected BloomFilter[] bloomFilters;
-    private MultiComparator btreeCmp;
+    protected MultiComparator btreeCmp;
     protected int numberOfTrees;
     protected SearchPredicate rtreeSearchPredicate;
     protected RangePredicate btreeRangePredicate;
@@ -63,6 +64,7 @@ public abstract class LSMRTreeAbstractCursor extends EnforcedIndexCursor impleme
     protected ILSMHarness lsmHarness;
     protected boolean foundNext;
     protected final ILSMIndexOperationContext opCtx;
+    protected ISearchOperationCallback searchCallback;
     protected List<ILSMComponent> operationalComponents;
     protected long[] hashes = BloomFilter.createHashArray();
 
@@ -86,6 +88,7 @@ public abstract class LSMRTreeAbstractCursor extends EnforcedIndexCursor impleme
         operationalComponents = lsmInitialState.getOperationalComponents();
         lsmHarness = lsmInitialState.getLSMHarness();
         numberOfTrees = operationalComponents.size();
+        searchCallback = lsmInitialState.getSearchOperationCallback();
 
         int numComponenets = operationalComponents.size();
         if (rtreeCursors == null || rtreeCursors.length != numComponenets) {
@@ -180,4 +183,9 @@ public abstract class LSMRTreeAbstractCursor extends EnforcedIndexCursor impleme
         return frameTuple;
     }
 
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index d485f64..c79735f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -31,6 +31,7 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
 
     private int currentCursor;
     private final PermutingTupleReference btreeTuple;
+    private boolean resultOfsearchCallbackProceed = false;
 
     public LSMRTreeSearchCursor(ILSMIndexOperationContext opCtx, int[] buddyBTreeFields) {
         super(opCtx);
@@ -96,6 +97,8 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
             while (rtreeCursors[currentCursor].hasNext()) {
                 rtreeCursors[currentCursor].next();
                 ITupleReference currentTuple = rtreeCursors[currentCursor].getTuple();
+                // Call proceed() to do necessary operations before returning this tuple.
+                resultOfsearchCallbackProceed = searchCallback.proceed(currentTuple);
                 btreeTuple.reset(rtreeCursors[currentCursor].getTuple());
                 boolean killerTupleFound = false;
                 for (int i = 0; i < currentCursor && !killerTupleFound; i++) {
@@ -138,4 +141,9 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
         searchNextCursor();
     }
 
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return resultOfsearchCallbackProceed;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
index 77bf58e..449c711 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
@@ -173,4 +173,9 @@ public class LSMRTreeWithAntiMatterTuplesFlushCursor extends EnforcedIndexCursor
         return null;
     }
 
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 4547063..094acbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -59,6 +59,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
     private int numMemoryComponents;
     private boolean open;
     protected ISearchOperationCallback searchCallback;
+    private boolean resultOfsearchCallBackProceed = false;
 
     public LSMRTreeWithAntiMatterTuplesSearchCursor(ILSMIndexOperationContext opCtx) {
         this(opCtx, false);
@@ -150,7 +151,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                     // reconcile() and complete() can be added later after considering the semantics.
 
                     // Call proceed() to do necessary operations before returning this tuple.
-                    searchCallback.proceed(currentTuple);
+                    resultOfsearchCallBackProceed = searchCallback.proceed(currentTuple);
                     if (searchMemBTrees(currentTuple, currentCursor)) {
                         // anti-matter tuple is NOT found
                         foundNext = true;
@@ -169,7 +170,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                 // reconcile() and complete() can be added later after considering the semantics.
 
                 // Call proceed() to do necessary operations before returning this tuple.
-                searchCallback.proceed(diskRTreeTuple);
+                resultOfsearchCallBackProceed = searchCallback.proceed(diskRTreeTuple);
                 if (searchMemBTrees(diskRTreeTuple, numMemoryComponents)) {
                     // anti-matter tuple is NOT found
                     foundNext = true;
@@ -185,7 +186,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                 // reconcile() and complete() can be added later after considering the semantics.
                 // Call proceed() to do necessary operations before returning this tuple.
                 // Since in-memory components don't exist, we can skip searching in-memory B-Trees.
-                searchCallback.proceed(diskRTreeTuple);
+                resultOfsearchCallBackProceed = searchCallback.proceed(diskRTreeTuple);
                 foundNext = true;
                 frameTuple = diskRTreeTuple;
                 return true;
@@ -311,4 +312,9 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
             }
         }
     }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return resultOfsearchCallBackProceed;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 0a4d2a7..46727cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -44,12 +44,26 @@ public class RTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
     protected final boolean retainMissing;
     protected final IMissingWriterFactory missingWriterFactory;
     protected final ISearchOperationCallbackFactory searchCallbackFactory;
+    protected boolean appendOpCallbackProceedResult;
+    protected byte[] searchCallbackProceedResultFalseValue;
+    protected byte[] searchCallbackProceedResultTrueValue;
 
     public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc, int[] keyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
             int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+        this(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
+                retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
+                maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+    }
+
+    public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc, int[] keyFields,
+            boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
+            int[] maxFilterFieldIndexes, boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -63,6 +77,9 @@ public class RTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
         this.outRecDescs[0] = outRecDesc;
+        this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
+        this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
+        this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
     }
 
     @Override
@@ -71,6 +88,7 @@ public class RTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
         return new RTreeSearchOperatorNodePushable(ctx, partition,
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), keyFields, minFilterFieldIndexes,
                 maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, appendIndexFilter);
+                searchCallbackFactory, appendIndexFilter, appendOpCallbackProceedResult,
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 07b6a60..886285c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -42,8 +42,22 @@ public class RTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePush
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             boolean appendIndexFilter) throws HyracksDataException {
+        this(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
+                null);
+    }
+
+    public RTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
+            int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+            IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
+            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
+            boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                searchCallbackProceedResultTrueValue);
         if (keyFields != null && keyFields.length > 0) {
             searchKey = new PermutingFrameTupleReference();
             searchKey.setFieldPermutation(keyFields);


[10/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-09.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-09.plan
new file mode 100644
index 0000000..cd96f94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-09.plan
@@ -0,0 +1,53 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- SORT_GROUP_BY[$$17]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+                -- PRE_CLUSTERED_GROUP_BY[$$32]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- UNION_ALL  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SPLIT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |UNPARTITIONED|
+                                                            -- UNNEST  |UNPARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
new file mode 100644
index 0000000..1264620
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$43(ASC) ]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$30]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$30(ASC), $$36(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- RTREE_SEARCH  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/keep-datetime-local-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/keep-datetime-local-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/keep-datetime-local-index-only.plan
new file mode 100644
index 0000000..fc39e6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/keep-datetime-local-index-only.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$48(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$50]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$50(ASC), $$23(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
+                          -- UNION_ALL  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search-index-only.plan
new file mode 100644
index 0000000..6044ccf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search-index-only.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SPLIT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-index-only.plan
new file mode 100644
index 0000000..648d3b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-index-only.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open-index-only.plan
new file mode 100644
index 0000000..648d3b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open-index-only.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
new file mode 100644
index 0000000..6a471bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
@@ -0,0 +1,62 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$56]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- RTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- RTREE_SEARCH  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
new file mode 100644
index 0000000..b8aad26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.plan
@@ -0,0 +1,62 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$63]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$63(ASC), $$24(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- RTREE_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_04.plan
new file mode 100644
index 0000000..f3f4745
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_04.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- RTREE_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_05.plan
new file mode 100644
index 0000000..f3f4745
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_05.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- RTREE_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
new file mode 100644
index 0000000..008be34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
@@ -0,0 +1,57 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- RTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- RTREE_SEARCH  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
new file mode 100644
index 0000000..0bfec2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01.plan
new file mode 100644
index 0000000..3d668e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-01.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- UNION_ALL  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- RTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- RTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-02.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-02.plan
new file mode 100644
index 0000000..0ec3715
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-02.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$11(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-03.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-03.plan
new file mode 100644
index 0000000..2915dbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-03.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- UNION_ALL  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-04.plan
new file mode 100644
index 0000000..dd61417
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-04.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$17(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- RTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- RTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-05.plan
new file mode 100644
index 0000000..e98529f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-05.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- RTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-06.plan
new file mode 100644
index 0000000..42986f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-06.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$16(ASC) ]  |PARTITIONED|
+                -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- UNION_ALL  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- RTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-07.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-07.plan
new file mode 100644
index 0000000..e98529f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-07.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- RTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- RTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-08.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-08.plan
new file mode 100644
index 0000000..6370454
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-08.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-09.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-09.plan
new file mode 100644
index 0000000..e744ecb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-secondary-index-indexonly-plan-09.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$10(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- RTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-index-only.plan
new file mode 100644
index 0000000..322e753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-index-only.plan
@@ -0,0 +1,47 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SPLIT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- RTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open-index-only.plan
new file mode 100644
index 0000000..322e753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open-index-only.plan
@@ -0,0 +1,47 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SPLIT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- RTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[14/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 1f4676c..a82e780 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -46,10 +46,13 @@ import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Quadruple;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
@@ -65,7 +68,6 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBina
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -86,14 +88,22 @@ public class BTreeAccessMethod implements IAccessMethod {
         EQUAL
     }
 
-    private static final List<FunctionIdentifier> FUNC_IDENTIFIERS =
-            Collections.unmodifiableList(Arrays.asList(AlgebricksBuiltinFunctions.EQ, AlgebricksBuiltinFunctions.LE,
-                    AlgebricksBuiltinFunctions.GE, AlgebricksBuiltinFunctions.LT, AlgebricksBuiltinFunctions.GT));
+    // The second boolean value tells whether the given function generates false positive results.
+    // That is, this function can produce false positive results if it is set to true.
+    // In this case, an index-search alone cannot replace the given SELECT condition and
+    // that SELECT condition needs to be applied after the index-search to get the correct results.
+    // For B+Tree indexes, there are no false positive results unless the given index is a composite index.
+    private static final List<Pair<FunctionIdentifier, Boolean>> FUNC_IDENTIFIERS = Collections
+            .unmodifiableList(Arrays.asList(new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.EQ, false),
+                    new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.LE, false),
+                    new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.GE, false),
+                    new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.LT, false),
+                    new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.GT, false)));
 
     public static final BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
 
     @Override
-    public List<FunctionIdentifier> getOptimizableFunctions() {
+    public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions() {
         return FUNC_IDENTIFIERS;
     }
 
@@ -123,59 +133,124 @@ public class BTreeAccessMethod implements IAccessMethod {
     public boolean applySelectPlanTransformation(List<Mutable<ILogicalOperator>> afterSelectRefs,
             Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree, Index chosenIndex,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
-        SelectOperator select = (SelectOperator) selectRef.getValue();
-        Mutable<ILogicalExpression> conditionRef = select.getCondition();
+        SelectOperator selectOp = (SelectOperator) selectRef.getValue();
+        Mutable<ILogicalExpression> conditionRef = selectOp.getCondition();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) conditionRef.getValue();
 
-        ILogicalOperator primaryIndexUnnestOp =
-                createSecondaryToPrimaryPlan(conditionRef, subTree, null, chosenIndex, analysisCtx,
-                        AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
-                                subTree.getDataSourceRef().getValue(), afterSelectRefs),
-                        false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
-                                .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                        context);
+        // Check whether assign (unnest) operator exists before the select operator
+        Mutable<ILogicalOperator> assignBeforeSelectOpRef =
+                subTree.getAssignsAndUnnestsRefs().isEmpty() ? null : subTree.getAssignsAndUnnestsRefs().get(0);
+        ILogicalOperator assignBeforeSelectOp = null;
+        if (assignBeforeSelectOpRef != null) {
+            assignBeforeSelectOp = assignBeforeSelectOpRef.getValue();
+        }
 
-        if (primaryIndexUnnestOp == null) {
+        Dataset dataset = subTree.getDataset();
+
+        // To check whether the given plan is an index-only plan.
+        // index-only plan possible?
+        boolean isIndexOnlyPlan = false;
+
+        // Whether a verification is required after this secondary index search.
+        // In other words, can the chosen method generate any false positive results?
+        // Currently, for the B+ Tree index, there cannot be any false positive results unless it's a composite index.
+        boolean requireVerificationAfterSIdxSearch = false;
+
+        Pair<Boolean, Boolean> functionFalsePositiveCheck =
+                AccessMethodUtils.canFunctionGenerateFalsePositiveResultsUsingIndex(funcExpr, FUNC_IDENTIFIERS);
+        if (functionFalsePositiveCheck.first) {
+            // An index-utilizable function found? then, get the info about false positive results generation.
+            requireVerificationAfterSIdxSearch = functionFalsePositiveCheck.second;
+        } else {
             return false;
         }
-        Mutable<ILogicalOperator> opRef =
-                subTree.getAssignsAndUnnestsRefs().isEmpty() ? null : subTree.getAssignsAndUnnestsRefs().get(0);
-        ILogicalOperator op = null;
-        if (opRef != null) {
-            op = opRef.getValue();
+
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo =
+                new Quadruple<>(isIndexOnlyPlan, false, requireVerificationAfterSIdxSearch, false);
+
+        if (dataset.getDatasetType() == DatasetType.INTERNAL && !chosenIndex.isPrimaryIndex()) {
+            AccessMethodUtils.indexOnlyPlanCheck(afterSelectRefs, selectRef, subTree, null, chosenIndex, analysisCtx,
+                    context, indexOnlyPlanInfo);
+            isIndexOnlyPlan = indexOnlyPlanInfo.getFirst();
         }
-        // Generate new select using the new condition.
+
+        // Sets the result of index-only plan check into AccessMethodAnalysisContext.
+        analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
+
+        // Transform the current path to the path that is utilizing the corresponding indexes
+        ILogicalOperator primaryIndexUnnestOp = createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
+                subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
+                AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
+                        afterSelectRefs),
+                false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
+                        .getExecutionMode() == ExecutionMode.UNPARTITIONED,
+                context, null);
+
+        if (primaryIndexUnnestOp == null) {
+            return false;
+        }
+
+        // Generate new path using the new condition.
         if (conditionRef.getValue() != null) {
-            select.getInputs().clear();
-            if (op != null) {
-                subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-                select.getInputs().add(new MutableObject<>(op));
+            if (assignBeforeSelectOp != null) {
+                if (isIndexOnlyPlan && dataset.getDatasetType() == DatasetType.INTERNAL) {
+                    // Case 1: index-only plan
+                    // The whole plan is now changed. Replace the current path with the given new plan.
+                    // Gets the revised dataSourceRef operator
+                    //  - a secondary index-search that generates trustworthy results.
+                    ILogicalOperator dataSourceRefOp =
+                            AccessMethodUtils.findDataSourceFromIndexUtilizationPlan(primaryIndexUnnestOp);
+
+                    if (dataSourceRefOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                        subTree.getDataSourceRef().setValue(dataSourceRefOp);
+                    }
+
+                    // Replace the current operator with the newly created operator.
+                    selectRef.setValue(primaryIndexUnnestOp);
+                } else {
+                    // Case 2: Non-index only plan case
+                    // Right now, the order of operators is: select <- assign <- unnest-map (primary index look-up).
+                    selectOp.getInputs().clear();
+                    subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
+                    selectOp.getInputs().add(new MutableObject<ILogicalOperator>(assignBeforeSelectOp));
+                }
             } else {
-                select.getInputs().add(new MutableObject<>(primaryIndexUnnestOp));
+                // A secondary-index-only plan without any assign cannot exist. This is a non-index only plan.
+                selectOp.getInputs().clear();
+                selectOp.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
             }
         } else {
+            // All condition is now gone. UNNEST-MAP can replace the SELECT operator.
             ((AbstractLogicalOperator) primaryIndexUnnestOp).setExecutionMode(ExecutionMode.PARTITIONED);
-            if (op != null) {
+            if (assignBeforeSelectOp != null) {
                 subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-                selectRef.setValue(op);
+                selectRef.setValue(assignBeforeSelectOp);
             } else {
                 selectRef.setValue(primaryIndexUnnestOp);
             }
         }
+
         return true;
     }
 
     @Override
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
-            boolean hasGroupBy) throws AlgebricksException {
+    public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
+            OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean hasGroupBy) throws AlgebricksException {
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
         Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
-        // Determine if the index is applicable on the left or right side
-        // (if both, we arbitrarily prefer the left side).
-        Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
-        OptimizableOperatorSubTree indexSubTree;
-        OptimizableOperatorSubTree probeSubTree;
+
+        AbstractFunctionCallExpression funcExpr = null;
+        if (conditionRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            funcExpr = (AbstractFunctionCallExpression) conditionRef.getValue();
+        }
+
+        Dataset dataset = analysisCtx.getIndexDatasetMap().get(chosenIndex);
+
+        // Determine if the index is applicable on the right (inner) side.
+        OptimizableOperatorSubTree indexSubTree = null;
+        OptimizableOperatorSubTree probeSubTree = null;
         // We assume that the left subtree is the outer branch and the right subtree is the inner branch.
         // This assumption holds true since we only use an index from the right subtree.
         // The following is just a sanity check.
@@ -189,42 +264,41 @@ public class BTreeAccessMethod implements IAccessMethod {
 
         LogicalVariable newNullPlaceHolderVar = null;
         if (isLeftOuterJoin) {
-            //get a new null place holder variable that is the first field variable of the primary key
-            //from the indexSubTree's datasourceScanOp
+            // Gets a new null place holder variable that is the first field variable of the primary key
+            // from the indexSubTree's datasourceScanOp.
             newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
         }
 
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, indexSubTree, probeSubTree,
-                chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context);
-        if (primaryIndexUnnestOp == null) {
+        boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
+                chosenIndex, analysisCtx, context, funcExpr, FUNC_IDENTIFIERS);
+        if (!canContinue) {
             return false;
         }
 
-        if (isLeftOuterJoin && hasGroupBy) {
-            //reset the null place holder variable
-            AccessMethodUtils.resetLOJNullPlaceholderVariableInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
-        }
+        ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
+                indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
+                isLeftOuterJoin, true, context, newNullPlaceHolderVar);
 
-        // If there are conditions left, add a new select operator on top.
-        indexSubTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-        if (conditionRef.getValue() != null) {
-            SelectOperator topSelect = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
-            topSelect.getInputs().add(indexSubTree.getRootRef());
-            topSelect.setExecutionMode(ExecutionMode.LOCAL);
-            context.computeAndSetTypeEnvironmentForOperator(topSelect);
-            // Replace the original join with the new subtree rooted at the select op.
-            joinRef.setValue(topSelect);
-        } else {
-            joinRef.setValue(indexSubTree.getRootRef().getValue());
+        if (indexSearchOp == null) {
+            return false;
         }
-        return true;
+
+        return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, analysisCtx,
+                context, isLeftOuterJoin, hasGroupBy, indexSearchOp, newNullPlaceHolderVar, conditionRef, dataset);
     }
 
+    /**
+     * Creates an index utilization plan optimization - in case of an index-only select plan:
+     * union < project < select < assign? < unnest-map(pidx) < split < unnest-map(sidx) < assign? < datasource-scan
+     * ..... < project < ................................... <
+     */
     @Override
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
-            IOptimizationContext context) throws AlgebricksException {
+    public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
         Dataset dataset = indexSubTree.getDataset();
         ARecordType recordType = indexSubTree.getRecordType();
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -233,10 +307,20 @@ public class BTreeAccessMethod implements IAccessMethod {
                 (AbstractDataSourceOperator) indexSubTree.getDataSourceRef().getValue();
         List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.getIndexExprsFromIndexExprsAndVars(chosenIndex);
         int numSecondaryKeys = analysisCtx.getNumberOfMatchedKeys(chosenIndex);
+
+        // Whether the given plan is an index-only plan or not.
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
+        boolean isIndexOnlyPlan = indexOnlyPlanInfo.getFirst();
+
+        // We only apply index-only plan for an internal dataset.
+        boolean generateInstantTrylockResultFromIndexSearch = false;
+        if (dataset.getDatasetType() == DatasetType.INTERNAL && isIndexOnlyPlan) {
+            generateInstantTrylockResultFromIndexSearch = true;
+        }
+
         // List of function expressions that will be replaced by the secondary-index search.
         // These func exprs will be removed from the select condition at the very end of this method.
         Set<ILogicalExpression> replacedFuncExprs = new HashSet<>();
-
         // Info on high and low keys for the BTree search predicate.
         ILogicalExpression[] lowKeyExprs = new ILogicalExpression[numSecondaryKeys];
         ILogicalExpression[] highKeyExprs = new ILogicalExpression[numSecondaryKeys];
@@ -257,6 +341,9 @@ public class BTreeAccessMethod implements IAccessMethod {
         BitSet setHighKeys = new BitSet(numSecondaryKeys);
         // Go through the func exprs listed as optimizable by the chosen index,
         // and formulate a range predicate on the secondary-index keys.
+
+        // Checks whether a type casting happened from a real (FLOAT, DOUBLE) value to an INT value
+        // since we have a round issue when dealing with LT(<) OR GT(>) operator.
         for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
             // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
             IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprIndex.first);
@@ -268,10 +355,16 @@ public class BTreeAccessMethod implements IAccessMethod {
             if (keyPos < 0) {
                 throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR);
             }
+            // returnedSearchKeyExpr contains a pair of search expression.
+            // The second expression will not be null only if we are creating an EQ search predicate
+            // with a FLOAT or a DOUBLE constant that will be fed into an INTEGER index.
+            // This is required because of type-casting. Refer to AccessMethodUtils.createSearchKeyExpr for details.
             IAType indexedFieldType = chosenIndex.getKeyFieldTypes().get(keyPos);
-            Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(chosenIndex,
-                    optFuncExpr, indexedFieldType, indexSubTree, probeSubTree);
+            Triple<ILogicalExpression, ILogicalExpression, Boolean> returnedSearchKeyExpr =
+                    AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexedFieldType, probeSubTree);
             ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
+            ILogicalExpression searchKeyEQExpr = null;
+            boolean realTypeConvertedToIntegerType = returnedSearchKeyExpr.third;
             if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 constantAtRuntimeExpressions[keyPos] = searchKeyExpr;
                 constAtRuntimeExprVars[keyPos] = context.newVar();
@@ -283,10 +376,13 @@ public class BTreeAccessMethod implements IAccessMethod {
                 return null;
             }
 
-            // checks whether a type casting happened from a real (FLOAT, DOUBLE) value to an INT value
-            // since we have a round issues when dealing with LT(<) OR GT(>) operator.
-            boolean realTypeConvertedToIntegerType = returnedSearchKeyExpr.second;
+            if (limit == LimitType.EQUAL && returnedSearchKeyExpr.second != null) {
+                // The given search predicate is EQ and
+                // we have two type-casted values from FLOAT or DOUBLE to an INT constant.
+                searchKeyEQExpr = returnedSearchKeyExpr.second;
+            }
 
+            // Deals with the non-enforced index case here.
             if (relaxLimitTypeToInclusive(chosenIndex, keyPos, realTypeConvertedToIntegerType)) {
                 if (limit == LimitType.HIGH_EXCLUSIVE) {
                     limit = LimitType.HIGH_INCLUSIVE;
@@ -300,7 +396,16 @@ public class BTreeAccessMethod implements IAccessMethod {
                     if (lowKeyLimits[keyPos] == null && highKeyLimits[keyPos] == null) {
                         lowKeyLimits[keyPos] = highKeyLimits[keyPos] = limit;
                         lowKeyInclusive[keyPos] = highKeyInclusive[keyPos] = true;
-                        lowKeyExprs[keyPos] = highKeyExprs[keyPos] = searchKeyExpr;
+                        if (searchKeyEQExpr == null) {
+                            // No type-casting was happened.
+                            lowKeyExprs[keyPos] = highKeyExprs[keyPos] = searchKeyExpr;
+                        } else {
+                            // We have two type-casted FLOAT or DOUBLE values to be fed into an INT index.
+                            // They contain the same value if their fraction value is 0.
+                            // Refer to AccessMethodUtils.createSearchKeyExpr() for more details.
+                            lowKeyExprs[keyPos] = searchKeyExpr;
+                            highKeyExprs[keyPos] = searchKeyEQExpr;
+                        }
                         setLowKeys.set(keyPos);
                         setHighKeys.set(keyPos);
                         isEqCondition = true;
@@ -444,8 +549,8 @@ public class BTreeAccessMethod implements IAccessMethod {
             highKeyInclusive[0] = true;
         }
 
-        // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index
-        // search.
+        // Here we generate vars and funcs for assigning the secondary-index keys to be fed into
+        // the secondary-index search.
         // List of variables for the assign.
         ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
         // List of variables and expressions for the assign.
@@ -477,7 +582,9 @@ public class BTreeAccessMethod implements IAccessMethod {
             } else {
                 // We are optimizing a join, place the assign op top of the probe subtree.
                 assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
+                assignSearchKeys.setExecutionMode(probeSubTree.getRootRef().getValue().getExecutionMode());
             }
+            context.computeAndSetTypeEnvironmentForOperator(assignSearchKeys);
             inputOp = assignSearchKeys;
         } else if (probeSubTree == null) {
             //nonpure case
@@ -496,36 +603,60 @@ public class BTreeAccessMethod implements IAccessMethod {
             inputOp = probeSubTree.getRoot();
         }
 
+        // Creates an unnest-map for the secondary index search.
+        // The result: SK, PK, [Optional - the result of an instantTrylock on PK]
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
-                metaRecordType, chosenIndex, inputOp, jobGenParams, context, false, retainInput, retainNull);
+                metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainMissing,
+                generateInstantTrylockResultFromIndexSearch);
 
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        AbstractUnnestMapOperator primaryIndexUnnestOp = null;
+        ILogicalOperator indexSearchOp = null;
 
         boolean isPrimaryIndex = chosenIndex.isPrimaryIndex();
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             // External dataset
-            UnnestMapOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp,
-                    dataset, recordType, secondaryIndexUnnestOp, context, retainInput, retainNull);
+            UnnestMapOperator externalDataAccessOp =
+                    AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp, dataset, recordType,
+                            metaRecordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainMissing);
             indexSubTree.getDataSourceRef().setValue(externalDataAccessOp);
             return externalDataAccessOp;
         } else if (!isPrimaryIndex) {
-            primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp, dataset, recordType,
-                    metaRecordType, secondaryIndexUnnestOp, context, true, retainInput, retainNull, false);
-
-            // Adds equivalence classes --- one equivalent class between a primary key
-            // variable and a record field-access expression.
-            EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp,
-                    dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
+            indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
+                    assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
+                    context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
+                    newMissingPlaceHolderForLOJ);
+
+            // Replaces the datasource scan with the new plan rooted at
+            // Get dataSourceRef operator -
+            // 1) unnest-map (PK, record) for a non-index only plan
+            // 2) unnest-map (SK, PK) for an index-only plan
+            if (isIndexOnlyPlan) {
+                // Index-only plan
+                ILogicalOperator dataSourceRefOp =
+                        AccessMethodUtils.findDataSourceFromIndexUtilizationPlan(indexSearchOp);
+
+                if (dataSourceRefOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP
+                        || dataSourceRefOp.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
+                    // Adds equivalence classes --- one equivalent class between a primary key
+                    // variable and a record field-access expression.
+                    EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(indexSearchOp,
+                            dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
+                }
+            } else {
+                // Non-indexonly plan cases
+                // Adds equivalence classes --- one equivalent class between a primary key
+                // variable and a record field-access expression.
+                EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(indexSearchOp,
+                        dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
+            }
         } else {
+            // Primary index search case
             List<Object> primaryIndexOutputTypes = new ArrayList<>();
             AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, metaRecordType, primaryIndexOutputTypes);
             List<LogicalVariable> scanVariables = dataSourceOp.getVariables();
 
-            // Checks whether the primary index search can replace the given
-            // SELECT condition.
-            // If so, condition will be set to null and eventually the SELECT
-            // operator will be removed.
+            // Checks whether the primary index search can replace the given SELECT condition.
+            // If so, the condition will be set to null and eventually the SELECT operator will be removed.
             // If not, we create a new condition based on remaining ones.
             if (!primaryIndexPostProccessingIsNeeded) {
                 List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<>();
@@ -534,7 +665,7 @@ public class BTreeAccessMethod implements IAccessMethod {
                 } catch (CompilationException e) {
                     return null;
                 }
-                // Generate new condition.
+                // Generates the new condition.
                 if (!remainingFuncExprs.isEmpty()) {
                     ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
                     conditionRef.setValue(pulledCond);
@@ -545,7 +676,7 @@ public class BTreeAccessMethod implements IAccessMethod {
 
             // Checks whether LEFT_OUTER_UNNESTMAP operator is required.
             boolean leftOuterUnnestMapRequired = false;
-            if (retainNull && retainInput) {
+            if (retainMissing && retainInput) {
                 leftOuterUnnestMapRequired = true;
             } else {
                 leftOuterUnnestMapRequired = false;
@@ -556,42 +687,41 @@ public class BTreeAccessMethod implements IAccessMethod {
                 // via the UnnestMapOperator's function arguments.
                 List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
                 jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
-                // An index search is expressed as an unnest-map over an
-                // index-search function.
+                // An index search is expressed as an unnest-map over an index-search function.
                 IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
                 UnnestingFunctionCallExpression primaryIndexSearchFunc =
                         new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
                 primaryIndexSearchFunc.setReturnsUniqueValues(true);
                 if (!leftOuterUnnestMapRequired) {
-                    primaryIndexUnnestOp = new UnnestMapOperator(scanVariables,
+                    indexSearchOp = new UnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    primaryIndexUnnestOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             true);
                 }
             } else {
                 if (!leftOuterUnnestMapRequired) {
-                    primaryIndexUnnestOp = new UnnestMapOperator(scanVariables,
+                    indexSearchOp = new UnnestMapOperator(scanVariables,
                             ((UnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    primaryIndexUnnestOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
                             ((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
                             primaryIndexOutputTypes, true);
                 }
             }
 
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
+            indexSearchOp.getInputs().add(new MutableObject<>(inputOp));
 
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
-            EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp, scanVariables,
-                    recordType, metaRecordType, dataset, context);
+            EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(indexSearchOp, scanVariables, recordType,
+                    metaRecordType, dataset, context);
         }
 
-        return primaryIndexUnnestOp;
+        return indexSearchOp;
     }
 
     private int createKeyVarsAndExprs(int numKeys, LimitType[] keyLimits, ILogicalExpression[] searchKeyExprs,
@@ -706,6 +836,13 @@ public class BTreeAccessMethod implements IAccessMethod {
     }
 
     private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType) {
+        // For a non-enforced index or an enforced index that stores a casted value on the given index,
+        // we need to apply the following transformation.
+        // For an index on a closed field, this transformation is not necessary since the value between
+        // the index and the actual record match.
+        //
+        // Check AccessMethodUtils.createSearchKeyExpr for more details.
+        //
         // If a DOUBLE or FLOAT constant is converted to an INT type value,
         // we need to check a corner case where two real values are located between an INT value.
         // For example, for the following query,
@@ -721,10 +858,7 @@ public class BTreeAccessMethod implements IAccessMethod {
         // Therefore, we convert LT(<) to LE(<=) and GT(>) to GE(>=) to find candidates.
         // This does not change the result of an actual comparison since this conversion is only applied
         // for finding candidates from an index.
-        //
-        // We also need to do this for a non-enforced index that overrides key field type (for a numeric type)
-
-        if (realTypeConvertedToIntegerType) {
+        if (chosenIndex.isEnforced() && realTypeConvertedToIntegerType) {
             return true;
         }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index 870b425..559f336 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -23,9 +23,11 @@ import java.util.List;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -41,9 +43,10 @@ public interface IAccessMethod extends Comparable<IAccessMethod> {
 
     /**
      * @return A list of function identifiers that are optimizable by this
-     *         access method.
+     *         access method. Also, the second boolean tells whether that
+     *         function can generate a false-positive result.
      */
-    public List<FunctionIdentifier> getOptimizableFunctions();
+    public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions();
 
     /**
      * Analyzes the arguments of a given optimizable funcExpr to see if this
@@ -86,20 +89,22 @@ public interface IAccessMethod extends Comparable<IAccessMethod> {
             Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree, Index chosenIndex,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException;
 
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
-            IOptimizationContext context) throws AlgebricksException;
+    public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException;
 
     /**
      * Applies the plan transformation to use chosenIndex to optimize a join query.
      * In the case of a LeftOuterJoin, there may or may not be a needed groupby operation
      * If there is, we will need to include it in the transformation
      */
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
-            boolean hasGroupBy) throws AlgebricksException;
+    public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
+            OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean hasGroupBy) throws AlgebricksException;
 
     /**
      * Analyzes expr to see whether it is optimizable by the given concrete index.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 1171ae5..401ea23 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.optimizer.rules.am;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -48,33 +49,37 @@ import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 /**
  * This rule optimizes a join with secondary indexes into an indexed nested-loop join.
- * Matches the following operator pattern:
- * (join) <-- (select)? <-- (assign | unnest)+ <-- (datasource scan)
- * <-- (select)? <-- (assign | unnest)+ <-- (datasource scan | unnest-map)
- * The order of the join inputs matters (left becomes the outer relation and right becomes the inner relation).
+ * This rule matches the following operator pattern:
+ * join <-- select? <-- (assign|unnest)+ <-- (datasource scan|unnest-map)
+ * The order of the join inputs matters (left branch - outer relation, right branch - inner relation).
  * This rule tries to utilize an index on the inner relation.
  * If that's not possible, it stops transforming the given join into an index-nested-loop join.
- * Replaces the above pattern with the following simplified plan:
- * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest(index search)) <-- (assign) <-- (A)
- * (A) <-- (datasource scan | unnest-map)
- * The sort is optional, and some access methods may choose not to sort.
+ *
+ * This rule replaces the above pattern with the following simplified plan:
+ * select <-- assign+ <-- unnest-map(pidx) <-- sort <-- unnest-map(sidx) <-- assign+ <-- (datasource scan|unnest-map)
+ * The sorting PK process is optional, and some access methods may choose not to sort.
  * Note that for some index-based optimizations we do not remove the triggering
  * condition from the join, since the secondary index may only act as a filter, and the
  * final verification must still be done with the original join condition.
+ *
  * The basic outline of this rule is:
  * 1. Match operator pattern.
  * 2. Analyze join condition to see if there are optimizable functions (delegated to IAccessMethods).
  * 3. Check metadata to see if there are applicable indexes.
  * 4. Choose an index to apply (for now only a single index will be chosen).
  * 5. Rewrite plan using index (delegated to IAccessMethods).
+ *
  * For left-outer-join, additional patterns are checked and additional treatment is needed as follows:
- * 1. First it checks if there is a groupByOp above the join: (groupby) <-- (leftouterjoin)
+ * 1. First it checks if there is a groupByOp above the join: groupby <-- leftouterjoin
  * 2. Inherently, only the right-subtree of the lojOp can be used as indexSubtree.
- * So, the right-subtree must have at least one applicable index on join field(s)
+ * So, the right-subtree must have at least one applicable index on join field(s).
  * 3. If there is a groupByOp, the null placeholder variable introduced in groupByOp should be taken care of correctly.
  * Here, the primary key variable from datasourceScanOp replaces the introduced null placeholder variable.
- * If the primary key is composite key, then the first variable of the primary key variables becomes the
+ * If the primary key is a composite key, then the first variable of the primary key variables becomes the
  * null place holder variable. This null placeholder variable works for all three types of indexes.
+ *
+ * If the inner-branch can be transformed as an index-only plan, this rule creates an index-only-plan path
+ * that is similar to one described in IntroduceSelectAccessMethod Rule.
  */
 public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethodRule {
 
@@ -84,10 +89,10 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
     protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
     protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
     protected IVariableTypeEnvironment typeEnvironment = null;
-    protected boolean isLeftOuterJoin = false;
     protected boolean hasGroupBy = true;
+    protected List<Mutable<ILogicalOperator>> afterJoinRefs = null;
 
-    // Register access methods.
+    // Registers access methods.
     protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
 
     static {
@@ -97,8 +102,8 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
     }
 
     /**
-     * Recursively check the given plan from the root operator to transform a plan
-     * with JOIN or LEFT-OUTER-JOIN operator into an index-utilized plan.
+     * Recursively checks the given plan from the root operator to transform a plan
+     * with INNERJOIN or LEFT-OUTER-JOIN operator into an index-utilized plan.
      */
 
     @Override
@@ -114,7 +119,7 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             return false;
         }
 
-        // Check whether this operator is the root, which is DISTRIBUTE_RESULT or SINK since
+        // Checks whether this operator is the root, which is DISTRIBUTE_RESULT or SINK since
         // we start the process from the root operator.
         if (op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT
                 && op.getOperatorTag() != LogicalOperatorTag.SINK
@@ -127,7 +132,8 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             return false;
         }
 
-        // Recursively check the given plan whether the desired pattern exists in it.
+        afterJoinRefs = new ArrayList<>();
+        // Recursively checks the given plan whether the desired pattern exists in it.
         // If so, try to optimize the plan.
         boolean planTransformed = checkAndApplyJoinTransformation(opRef, context);
 
@@ -227,11 +233,11 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         joinRef = null;
         joinOp = null;
         joinCond = null;
-        isLeftOuterJoin = false;
+        afterJoinRefs = null;
     }
 
     /**
-     * Recursively traverse the given plan and check whether a INNERJOIN or LEFTOUTERJOIN operator exists.
+     * Recursively traverses the given plan and check whether a INNERJOIN or LEFTOUTERJOIN operator exists.
      * If one is found, maintain the path from the root to the given join operator and
      * optimize the path from the given join operator to the EMPTY_TUPLE_SOURCE operator
      * if it is not already optimized.
@@ -241,6 +247,10 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         boolean joinFoundAndOptimizationApplied;
 
+        // Adds the current operator to the operator list that contains operators after a join operator
+        // in case there is a descendant join operator and it could be transformed first.
+        afterJoinRefs.add(opRef);
+
         // Recursively check the plan and try to optimize it. We first check the children of the given operator
         // to make sure an earlier join in the path is optimized first.
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
@@ -250,32 +260,39 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             }
         }
 
-        // Check the current operator pattern to see whether it is a JOIN or not.
+        // Now, we are sure that transformation attempts for earlier joins have been failed.
+        // Checks the current operator pattern to see whether it is a JOIN or not.
         boolean isThisOpInnerJoin = isInnerJoin(op);
         boolean isThisOpLeftOuterJoin = isLeftOuterJoin(op);
         boolean isParentOpGroupBy = hasGroupBy;
 
         Mutable<ILogicalOperator> joinRefFromThisOp = null;
         AbstractBinaryJoinOperator joinOpFromThisOp = null;
-
+        // operators that need to be removed from the afterJoinRefs list.
+        Mutable<ILogicalOperator> opRefRemove = opRef;
         if (isThisOpInnerJoin) {
-            // Set join operator.
+            // Sets the join operator.
             joinRef = opRef;
             joinOp = (InnerJoinOperator) op;
             joinRefFromThisOp = opRef;
             joinOpFromThisOp = (InnerJoinOperator) op;
         } else if (isThisOpLeftOuterJoin) {
-            // Set left-outer-join op.
-            // The current operator is GROUP and the child of this op is LEFTOUERJOIN.
+            // Sets the left-outer-join operator.
+            // The current operator is GROUP and the child of this operator is LEFTOUERJOIN.
             joinRef = op.getInputs().get(0);
             joinOp = (LeftOuterJoinOperator) joinRef.getValue();
             joinRefFromThisOp = op.getInputs().get(0);
             joinOpFromThisOp = (LeftOuterJoinOperator) joinRefFromThisOp.getValue();
+
+            // Group-by should not be removed at this point since the given left-outer-join can be transformed.
+            opRefRemove = op.getInputs().get(0);
         }
+        afterJoinRefs.remove(opRefRemove);
 
-        // For a JOIN case, try to transform the given plan.
+        // For a JOIN case, tries to transform the given plan.
         if (isThisOpInnerJoin || isThisOpLeftOuterJoin) {
-            // Restore the information from this operator since it might have been be set to null
+
+            // Restores the information from this operator since it might have been be set to null
             // if there are other join operators in the earlier path.
             joinRef = joinRefFromThisOp;
             joinOp = joinOpFromThisOp;
@@ -294,14 +311,14 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                 analyzedAMs = new HashMap<>();
             }
 
-            // Check the condition of JOIN operator is a function call since only function call can be transformed
-            // using available indexes. If so, initialize the subtree information that will be used later to decide
+            // Checks the condition of JOIN operator is a function call since only function call can be transformed
+            // using available indexes. If so, initializes the subtree information that will be used later to decide
             // whether the given plan is truly optimizable or not.
             if (continueCheck && !checkJoinOpConditionAndInitSubTree(context)) {
                 continueCheck = false;
             }
 
-            // Analyze the condition of SELECT operator and initialize analyzedAMs.
+            // Analyzes the condition of SELECT operator and initializes analyzedAMs.
             // Check whether the function in the SELECT operator can be truly transformed.
             boolean matchInLeftSubTree = false;
             boolean matchInRightSubTree = false;
@@ -316,7 +333,7 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                 }
             }
 
-            // Find the dataset from the data-source and the record type of the dataset from the metadata.
+            // Finds the dataset from the data-source and the record type of the dataset from the metadata.
             // This will be used to find an applicable index on the dataset.
             boolean checkLeftSubTreeMetadata = false;
             boolean checkRightSubTreeMetadata = false;
@@ -338,11 +355,11 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                 }
                 fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
 
-                // Prune the access methods based on the function expression and access methods.
+                // Prunes the access methods based on the function expression and access methods.
                 pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
 
                 // If the right subtree (inner branch) has indexes, one of those indexes will be used.
-                // Remove the indexes from the outer branch in the optimizer's consideration list for this rule.
+                // Removes the indexes from the outer branch in the optimizer's consideration list for this rule.
                 pruneIndexCandidatesFromOuterBranch(analyzedAMs);
 
                 // We are going to use indexes from the inner branch.
@@ -354,7 +371,16 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                 }
 
                 if (continueCheck) {
-                    // Apply plan transformation using chosen index.
+                    // Finds the field name of each variable in the sub-tree such as variables for order by.
+                    // This step is required when checking index-only plan.
+                    if (checkLeftSubTreeMetadata) {
+                        fillFieldNamesInTheSubTree(leftSubTree);
+                    }
+                    if (checkRightSubTreeMetadata) {
+                        fillFieldNamesInTheSubTree(rightSubTree);
+                    }
+
+                    // Applies the plan transformation using chosen index.
                     AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
 
                     // For LOJ with GroupBy, prepare objects to reset LOJ nullPlaceHolderVariable
@@ -363,7 +389,7 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                         analysisCtx.setLOJGroupbyOpRef(opRef);
                         ScalarFunctionCallExpression isNullFuncExpr =
                                 AccessMethodUtils.findLOJIsMissingFuncInGroupBy((GroupByOperator) opRef.getValue());
-                        analysisCtx.setLOJIsNullFuncInGroupBy(isNullFuncExpr);
+                        analysisCtx.setLOJIsMissingFuncInGroupBy(isNullFuncExpr);
                     }
 
                     Dataset indexDataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex.second);
@@ -376,9 +402,10 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                         return false;
                     }
 
-                    // Finally, try to apply plan transformation using chosen index.
-                    boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
-                            chosenIndex.second, analysisCtx, context, isThisOpLeftOuterJoin, isParentOpGroupBy);
+                    // Finally, tries to apply plan transformation using the chosen index.
+                    boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
+                            rightSubTree, chosenIndex.second, analysisCtx, context, isThisOpLeftOuterJoin,
+                            isParentOpGroupBy);
 
                     // If the plan transformation is successful, we don't need to traverse the plan
                     // any more, since if there are more JOIN operators, the next trigger on this plan
@@ -393,11 +420,19 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             joinOp = null;
         }
 
+        // Checked the given left-outer-join operator and it is not transformed. So, this group-by operator
+        // after the left-outer-join operator should be removed from the afterJoinRefs list
+        // since the current operator is a group-by operator.
+        if (isThisOpLeftOuterJoin) {
+            afterJoinRefs.remove(opRef);
+        }
+
         return false;
     }
 
     /**
-     * After the pattern is matched, check the condition and initialize the data sources from the both sub trees.
+     * After the pattern is matched, checks the condition and initializes the data source
+     * from the right (inner) sub tree.
      *
      * @throws AlgebricksException
      */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 546652b..7938f49 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -62,6 +62,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperat
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -79,7 +80,6 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-
         if (!checkIfRuleIsApplicable(opRef, context)) {
             return false;
         }
@@ -232,23 +232,23 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
             queue.addAll(descendantOp.getInputs());
         }
         if (hasSecondaryIndexMap && !primaryUnnestMapOps.isEmpty()) {
-            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context);
+            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context, false);
         }
     }
 
     private void propagateFilterToPrimaryIndex(List<UnnestMapOperator> primaryUnnestMapOps, IAType filterType,
-            IOptimizationContext context) throws AlgebricksException {
+            IOptimizationContext context, boolean isIndexOnlyPlan) throws AlgebricksException {
         for (UnnestMapOperator primaryOp : primaryUnnestMapOps) {
             Mutable<ILogicalOperator> assignOrOrderOrIntersect = primaryOp.getInputs().get(0);
-            Mutable<ILogicalOperator> intersectOrSort = assignOrOrderOrIntersect;
+            Mutable<ILogicalOperator> intersectOrSortOrSplit = assignOrOrderOrIntersect;
 
             if (assignOrOrderOrIntersect.getValue().getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-                intersectOrSort = assignOrOrderOrIntersect.getValue().getInputs().get(0);
+                intersectOrSortOrSplit = assignOrOrderOrIntersect.getValue().getInputs().get(0);
             }
 
-            switch (intersectOrSort.getValue().getOperatorTag()) {
+            switch (intersectOrSortOrSplit.getValue().getOperatorTag()) {
                 case INTERSECT:
-                    IntersectOperator intersect = (IntersectOperator) (intersectOrSort.getValue());
+                    IntersectOperator intersect = (IntersectOperator) (intersectOrSortOrSplit.getValue());
                     List<List<LogicalVariable>> filterVars = new ArrayList<>(intersect.getInputs().size());
                     for (Mutable<ILogicalOperator> mutableOp : intersect.getInputs()) {
                         ILogicalOperator child = mutableOp.getValue();
@@ -267,13 +267,30 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                         IntersectOperator intersectWithFilter =
                                 createIntersectWithFilter(outputFilterVars, filterVars, intersect);
 
-                        intersectOrSort.setValue(intersectWithFilter);
+                        intersectOrSortOrSplit.setValue(intersectWithFilter);
                         context.computeAndSetTypeEnvironmentForOperator(intersectWithFilter);
                         setPrimaryFilterVar(primaryOp, outputFilterVars.get(0), outputFilterVars.get(1), context);
                     }
                     break;
+                case SPLIT:
+                    if (!isIndexOnlyPlan) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                                intersectOrSortOrSplit.getValue().getOperatorTag().toString());
+                    }
+                    SplitOperator split = (SplitOperator) (intersectOrSortOrSplit.getValue());
+                    for (Mutable<ILogicalOperator> childOp : split.getInputs()) {
+                        ILogicalOperator child = childOp.getValue();
+                        while (!child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
+                            child = child.getInputs().get(0).getValue();
+                        }
+                        UnnestMapOperator unnestMap = (UnnestMapOperator) child;
+                        propagateFilterInSecondaryUnnsetMap(unnestMap, filterType, context);
+                        setPrimaryFilterVar(primaryOp, unnestMap.getPropagateIndexMinFilterVar(),
+                                unnestMap.getPropagateIndexMaxFilterVar(), context);
+                    }
+                    break;
                 case ORDER:
-                    ILogicalOperator child = intersectOrSort.getValue().getInputs().get(0).getValue();
+                    ILogicalOperator child = intersectOrSortOrSplit.getValue().getInputs().get(0).getValue();
                     if (child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
                         UnnestMapOperator secondaryMap = (UnnestMapOperator) child;
 
@@ -283,9 +300,10 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                                 secondaryMap.getPropagateIndexMaxFilterVar(), context);
                     }
                     break;
+
                 default:
                     throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
-                            intersectOrSort.getValue().getOperatorTag().toString());
+                            intersectOrSortOrSplit.getValue().getOperatorTag().toString());
             }
         }
     }
@@ -337,6 +355,7 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
             IOptimizationContext context, IAType filterType) throws AlgebricksException {
         List<UnnestMapOperator> primaryUnnestMapOps = new ArrayList<>();
         boolean hasSecondaryIndexMap = false;
+        boolean isIndexOnlyPlan = false;
         Queue<Mutable<ILogicalOperator>> queue = new LinkedList<>(op.getInputs());
         while (!queue.isEmpty()) {
             ILogicalOperator descendantOp = queue.poll().getValue();
@@ -356,6 +375,7 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                             primaryUnnestMapOps.add(unnestMapOp);
                         } else {
                             hasSecondaryIndexMap = true;
+                            isIndexOnlyPlan = unnestMapOp.getGenerateCallBackProceedResultVar();
                         }
                     }
                 }
@@ -363,7 +383,7 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
             queue.addAll(descendantOp.getInputs());
         }
         if (hasSecondaryIndexMap && !primaryUnnestMapOps.isEmpty()) {
-            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context);
+            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context, isIndexOnlyPlan);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index 7b8b906..5eb0fc6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -233,7 +233,7 @@ public class IntroducePrimaryIndexForAggregationRule implements IAlgebraicRewrit
             AbstractUnnestMapOperator primaryIndexUnnestOperator =
                     (AbstractUnnestMapOperator) AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                             metaRecordType, primaryIndex, scanOperator.getInputs().get(0).getValue(),
-                            newBTreeParameters, context, true, retainInput, false);
+                            newBTreeParameters, context, retainInput, false, false);
 
             // re-use the PK variables of the original scan operator
             primaryIndexUnnestOperator.getVariables().clear();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index d0e973f..27e5645 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -26,6 +26,8 @@ import java.util.Optional;
 import java.util.TreeMap;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -51,30 +53,62 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperat
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 /**
- * This rule optimizes simple selections with secondary or primary indexes. The use of an
- * index is expressed as an unnest-map over an index-search function which will be
+ * This rule optimizes simple selections with secondary or primary indexes.
+ * The use of an index is expressed as an UNNESTMAP operator over an index-search function which will be
  * replaced with the appropriate embodiment during codegen.
- * Matches the following operator patterns:
- * Standard secondary index pattern:
- * There must be at least one assign, but there may be more, e.g., when matching similarity-jaccard-check().
- * (select) <-- (assign | unnest)+ <-- (datasource scan)
- * Primary index lookup pattern:
- * Since no assign is necessary to get the primary key fields (they are already stored fields in the BTree tuples).
- * (select) <-- (datasource scan)
- * Replaces the above patterns with this plan:
- * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest-map(index search)) <-- (assign)
- * The sort is optional, and some access methods implementations may choose not to sort.
+ * This rule seeks to change the following patterns.
+ * For the secondary-index searches, a SELECT operator is followed by one or more ASSIGN / UNNEST operators.
+ * A DATASOURCE_SCAN operator should be placed before these operators.
+ * For the primary-index search, a SELECT operator is followed by DATASOURE_SCAN operator since no ASSIGN / UNNEST
+ * operator is required to get the primary key fields (they are already stored fields in the BTree tuples).
+ * If the above pattern is found, this rule replaces the pattern with the following pattern.
+ * If the given plan is both a secondary-index search and an index-only plan, it builds two paths.
+ * The left path has a UNIONALL operator at the top. And the original SELECT operator is followed. Also, the original
+ * ASSIGN / UNNEST operators are followed. Then, UNNEST-MAP for the primary-index-search is followed
+ * to fetch the record. Before that, a SPLIT operator is introduced. Before this, an UNNEST-MAP for
+ * the secondary-index-search is followed to conduct a secondary-index search. The search key (original ASSIGN/UNNEST)
+ * to the secondary-index-search (UNNEST-MAP) is placed before that.
+ * The right path has the above UNIONALL operator at the top. Then, possibly has optional SELECT and/or ASSIGN/UNNEST
+ * operators for the composite BTree or RTree search cases. Then, the above SPLIT operator is followed. Before the SPLIT
+ * operator, it shares the same operators with the left path.
+ * To be qualified as an index-only plan, there are two conditions.
+ * 1) Search predicate can be covered by a secondary index-search.
+ * 2) there are only PK and/or SK fields in the return clause.
+ * If the given query satisfies the above conditions, we call it an index-only plan.
+ * The benefit of the index-only plan is that we don't need to traverse the primary index
+ * after fetching SK, PK entries from a secondary index.
+ * The index-only plan works as follows.
+ * 1) During a secondary-index search, after fetching <SK, PK> pair that satisfies the given predicate,
+ * we try to get an instantTryLock on PK to verify that <SK, PK> is a valid pair.
+ * If it succeeds, the given <SK, PK> pair is trustworthy so that we can return this as a valid output.
+ * This tuple goes to the right path of UNIONALL operator since we don't need to traverse the primary index.
+ * If instantTryLock on PK fails, an operation on the PK record is ongoing, so we need to traverse
+ * the primary index to fetch the entire record and verify the search predicate. So, this <SK, PK> pair
+ * goes to the left path of UNIONALL operator to traverse the primary index.
+ * In the left path, we fetch the record using the given PK and fetch SK field and does SELECT verification.
+ * 2) A UNIONALL operator combines tuples from the left path and the right path and the rest of the plan continues.
+ * In an index-only plan, sort before the primary index-search is not required since we assume that
+ * the chances that a tuple (<SK, PK> pair) goes into the left path are low.
+ * If the given query plan is not an index-only plan, we call this plan as non-index only plan.
+ * In this case, the original plan will be transformed into the following pattern.
+ * The original SELECT operator is placed at the top. And the original ASSIGN / UNNEST operators are followed.
+ * An UNNEST-MAP that conducts the primary-index-search to fetch the primary keys are placed before that. An ORDER
+ * operator is placed to sort the primary keys before feed them into the primary-index. Then, an UNNEST-MAP is followed
+ * to conduct a secondary-index search. Then, the search key (ASSIGN / UNNEST) is followed.
+ * In this case, the sort is optional, and some access methods implementations may choose not to sort.
  * Note that for some index-based optimizations we do not remove the triggering
  * condition from the select, since the index may only acts as a filter, and the
  * final verification must still be done with the original select condition.
  * The basic outline of this rule is:
  * 1. Match operator pattern.
  * 2. Analyze select condition to see if there are optimizable functions (delegated to IAccessMethods).
- * 3. Check metadata to see if there are applicable indexes.
- * 4. Choose an index to apply (for now only a single index will be chosen).
- * 5. Rewrite plan using index (delegated to IAccessMethods).
- * If there are multiple secondary index access path available, we will use the intersection operator to get the
- * intersected primary key from all the secondary indexes. The detailed documentation is here
+ * 3. Check meta-data to see if there are applicable indexes.
+ * 4. Choose an index (or more indexes) to apply.
+ * 5. Rewrite the plan using index(es) (delegated to IAccessMethods).
+ * If multiple secondary index access paths are available, the optimizer uses the intersection operator
+ * to get the intersected primary key from all the chosen secondary indexes. In this case, we don't check
+ * whether the given plan is an index-only plan.
+ * The detailed documentation of intersecting multiple secondary indexes is here:
  * https://cwiki.apache.org/confluence/display/ASTERIXDB/Intersect+multiple+secondary+index
  */
 public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMethodRule {
@@ -169,26 +203,15 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
     }
 
     /**
-     * Construct all applicable secondary index-based access paths in the given selection plan and
-     * intersect them using INTERSECT operator to guide to the common primary index search.
-     * In case where the applicable index is one, we only construct one path.
+     * Constructs all applicable secondary index-based access paths in the given selection plan and
+     * intersects them using INTERSECT operator to guide to the common primary-index search.
+     * This method assumes that there are two or more secondary indexes in the given path.
      */
     private boolean intersectAllSecondaryIndexes(List<Pair<IAccessMethod, Index>> chosenIndexes,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context)
             throws AlgebricksException {
-        Pair<IAccessMethod, Index> chosenIndex = null;
-        Optional<Pair<IAccessMethod, Index>> primaryIndex =
-                chosenIndexes.stream().filter(pair -> pair.second.isPrimaryIndex()).findFirst();
         if (chosenIndexes.size() == 1) {
-            chosenIndex = chosenIndexes.get(0);
-        } else if (primaryIndex.isPresent()) {
-            // one primary + secondary indexes, choose the primary index directly.
-            chosenIndex = primaryIndex.get();
-        }
-        if (chosenIndex != null) {
-            AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
-            return chosenIndex.first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
-                    chosenIndex.second, analysisCtx, context);
+            throw CompilationException.create(ErrorCode.CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE);
         }
 
         // Intersect all secondary indexes, and postpone the primary index search.
@@ -197,12 +220,13 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
         List<ILogicalOperator> subRoots = new ArrayList<>();
         for (Pair<IAccessMethod, Index> pair : chosenIndexes) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(pair.first);
-            subRoots.add(pair.first.createSecondaryToPrimaryPlan(conditionRef, subTree, null, pair.second, analysisCtx,
+            subRoots.add(pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
+                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx,
                     AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
                             subTree.getDataSourceRef().getValue(), afterSelectRefs),
                     false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
                             .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                    context));
+                    context, null));
         }
         // Connect each secondary index utilization plan to a common intersect operator.
         ILogicalOperator primaryUnnestOp = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
@@ -212,6 +236,24 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
     }
 
     /**
+     * Checks whether the primary index exists among the applicable indexes and return it if is exists.
+     *
+     * @param chosenIndexes
+     * @return Pair<IAccessMethod, Index> for the primary index
+     *         null otherwise
+     * @throws AlgebricksException
+     */
+    private Pair<IAccessMethod, Index> fetchPrimaryIndexAmongChosenIndexes(
+            List<Pair<IAccessMethod, Index>> chosenIndexes) throws AlgebricksException {
+        Optional<Pair<IAccessMethod, Index>> primaryIndex =
+                chosenIndexes.stream().filter(pair -> pair.second.isPrimaryIndex()).findFirst();
+        if (primaryIndex.isPresent()) {
+            return primaryIndex.get();
+        }
+        return null;
+    }
+
+    /**
      * Connect each secondary index utilization plan to a common INTERSECT operator.
      */
     private ILogicalOperator connectAll2ndarySearchPlanWithIntersect(List<ILogicalOperator> subRoots,
@@ -352,9 +394,36 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
                 }
 
                 // Apply plan transformation using chosen index.
-                boolean res = intersectAllSecondaryIndexes(chosenIndexes, analyzedAMs, context);
-                context.addToDontApplySet(this, selectOp);
+                boolean res;
+
+                // Primary index applicable?
+                Pair<IAccessMethod, Index> chosenPrimaryIndex = fetchPrimaryIndexAmongChosenIndexes(chosenIndexes);
+                if (chosenPrimaryIndex != null) {
+                    AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenPrimaryIndex.first);
+                    res = chosenPrimaryIndex.first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
+                            chosenPrimaryIndex.second, analysisCtx, context);
+                    context.addToDontApplySet(this, selectRef.getValue());
+                } else if (chosenIndexes.size() == 1) {
+                    // Index-only plan possible?
+                    // Gets the analysis context for the given index.
+                    AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndexes.get(0).first);
+
+                    // Finds the field name of each variable in the sub-tree.
+                    fillFieldNamesInTheSubTree(subTree);
+
+                    // Finally, try to apply plan transformation using chosen index.
+                    res = chosenIndexes.get(0).first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
+                            chosenIndexes.get(0).second, analysisCtx, context);
+                    context.addToDontApplySet(this, selectRef.getValue());
+                } else {
+                    // Multiple secondary indexes applicable?
+                    res = intersectAllSecondaryIndexes(chosenIndexes, analyzedAMs, context);
+                    context.addToDontApplySet(this, selectRef.getValue());
+                }
 
+                // If the plan transformation is successful, we don't need to traverse
+                // the plan any more, since if there are more SELECT operators, the next
+                // trigger on this plan will find them.
                 if (res) {
                     OperatorPropertiesUtil.typeOpRec(opRef, context);
                     return res;
@@ -366,7 +435,7 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
             afterSelectRefs.add(opRef);
         }
 
-        // Clean the path after SELECT operator by removing the current operator in the list.
+        // Cleans the path after SELECT operator by removing the current operator in the list.
         afterSelectRefs.remove(opRef);
 
         return false;


[16/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
[ASTERIXDB-1972][COMP][RT][TX] index-only plan

- user model changes: no
- storage format changes: no
- interface changes: IAccessMethod, ILSMIndexOperationContext,
                     IIndexAccessor

Details:
- Implement an index-only plan
- Add a SET option that disables the index-only plan

Change-Id: Ifd5c9ab1cf2e4bedb7d8db582441919875e74d51
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1866
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Taewoo Kim <wa...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/c3c23574
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/c3c23574
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/c3c23574

Branch: refs/heads/master
Commit: c3c23574327d8c950820ea3e6539f4a1ade77fb3
Parents: 028537d
Author: Taewoo Kim <wa...@yahoo.com>
Authored: Thu Feb 15 21:34:32 2018 -0800
Committer: Taewoo Kim <wa...@gmail.com>
Committed: Fri Feb 16 11:03:38 2018 -0800

----------------------------------------------------------------------
 .../physical/BTreeSearchPOperator.java          |    3 +-
 .../physical/RTreeSearchPOperator.java          |    8 +-
 .../asterix/optimizer/base/RuleCollections.java |    3 +-
 .../am/AbstractIntroduceAccessMethodRule.java   |  183 +-
 .../rules/am/AccessMethodAnalysisContext.java   |   64 +-
 .../optimizer/rules/am/AccessMethodUtils.java   | 1858 +++++++++++++++++-
 .../optimizer/rules/am/BTreeAccessMethod.java   |  338 +++-
 .../optimizer/rules/am/IAccessMethod.java       |   25 +-
 .../rules/am/IntroduceJoinAccessMethodRule.java |  111 +-
 .../am/IntroduceLSMComponentFilterRule.java     |   42 +-
 ...IntroducePrimaryIndexForAggregationRule.java |    2 +-
 .../am/IntroduceSelectAccessMethodRule.java     |  143 +-
 .../rules/am/InvertedIndexAccessMethod.java     |  125 +-
 .../rules/am/OptimizableOperatorSubTree.java    |   67 +-
 .../optimizer/rules/am/RTreeAccessMethod.java   |  270 ++-
 .../data/nontagged/customerData2.json           |    6 +
 .../apache/asterix/api/common/APIFramework.java |    3 +-
 ...-pidx-with-join-btree-sidx_01-index-only.aql |   72 +
 ...rjoin-probe-pidx-with-join-btree-sidx_01.aql |    5 +-
 ...to-secondary-indexonly-plan-equi-join_01.aql |   69 +
 .../secondary-equi-join_04.sqlpp                |   44 +
 .../secondary-equi-join_05.sqlpp                |   42 +
 .../secondary-equi-join_06.sqlpp                |   43 +
 ...y-indexonly-plan-to-primary-equi-join_01.aql |   71 +
 ...to-secondary-indexonly-plan-equi-join_01.aql |   72 +
 ...to-secondary-indexonly-plan-equi-join_01.aql |   74 +
 .../secondary-self-equi-join-index-only.aql     |   65 +
 .../secondary-self-equi-join.aql                |    2 +-
 ...ondary-composite-index-indexonly-plan-01.aql |   64 +
 ...dexonly-plan-01-disable-indexonly-plan.sqlpp |   67 +
 ...tree-secondary-index-indexonly-plan-01.sqlpp |   64 +
 .../btree-secondary-index-indexonly-plan-02.aql |   65 +
 .../btree-secondary-index-indexonly-plan-03.aql |   65 +
 .../btree-secondary-index-indexonly-plan-04.aql |   67 +
 .../btree-secondary-index-indexonly-plan-05.aql |   67 +
 .../btree-secondary-index-indexonly-plan-06.aql |   67 +
 .../btree-secondary-index-indexonly-plan-07.aql |   67 +
 .../btree-secondary-index-indexonly-plan-08.aql |   67 +
 .../btree-secondary-index-indexonly-plan-09.aql |   50 +
 ...-pidx-with-join-rtree-sidx_01-index-only.aql |   75 +
 ...rjoin-probe-pidx-with-join-rtree-sidx_01.aql |    2 +-
 .../nonpure/keep-datetime-local-index-only.aql  |   74 +
 .../queries/nonpure/keep-datetime-local.aql     |    2 +-
 ...orders-composite-index-search-index-only.aql |   46 +
 .../queries/orders-composite-index-search.aql   |   18 +-
 .../queries/orders-index-search-index-only.aql  |   53 +
 .../orders-index-search-open-index-only.aql     |   51 +
 .../queries/orders-index-search-open.aql        |   20 +-
 .../optimizerts/queries/orders-index-search.aql |   20 +-
 .../rtree-index-join/issue730-index-only.aql    |   60 +
 .../queries/rtree-index-join/issue730.aql       |    6 +-
 ...-pidx-with-join-rtree-sidx_01-index-only.aql |   72 +
 ...rjoin-probe-pidx-with-join-rtree-sidx_01.aql |    5 +-
 .../spatial-intersect-point_04.aql              |   49 +
 .../spatial-intersect-point_05.aql              |   49 +
 .../spatial-self-intersect-point-index-only.aql |   64 +
 .../spatial-self-intersect-point.aql            |    2 +-
 ...dexonly-plan-01-disable-indexonly-plan.sqlpp |  114 ++
 ...tree-secondary-index-indexonly-plan-01.sqlpp |  111 ++
 .../rtree-secondary-index-indexonly-plan-02.aql |  111 ++
 .../rtree-secondary-index-indexonly-plan-03.aql |  111 ++
 .../rtree-secondary-index-indexonly-plan-04.aql |  114 ++
 .../rtree-secondary-index-indexonly-plan-05.aql |  114 ++
 .../rtree-secondary-index-indexonly-plan-06.aql |  114 ++
 .../rtree-secondary-index-indexonly-plan-07.aql |  114 ++
 .../rtree-secondary-index-indexonly-plan-08.aql |  114 ++
 .../rtree-secondary-index-indexonly-plan-09.aql |  112 ++
 .../rtree-secondary-index-index-only.aql        |   51 +
 .../rtree-secondary-index-open-index-only.aql   |   51 +
 .../queries/rtree-secondary-index-open.aql      |    3 +-
 .../queries/rtree-secondary-index.aql           |    5 +-
 .../udfs/query-ASTERIXDB-1019-index-only.aql    |   75 +
 .../queries/udfs/query-ASTERIXDB-1019.aql       |    2 +
 ...pidx-with-join-btree-sidx_01-index-only.plan |   50 +
 ...o-secondary-indexonly-plan-equi-join_01.plan |   45 +
 .../secondary-equi-join_04.plan                 |    8 +-
 .../secondary-equi-join_05.plan                 |    2 +-
 .../secondary-equi-join_06.plan                 |    8 +-
 ...-indexonly-plan-to-primary-equi-join_01.plan |   36 +
 ...o-secondary-indexonly-plan-equi-join_01.plan |   81 +
 ...o-secondary-indexonly-plan-equi-join_01.plan |   65 +
 .../secondary-self-equi-join-index-only.plan    |   47 +
 ...ndary-composite-index-indexonly-plan-01.plan |   31 +
 ...ndexonly-plan-01-disable-indexonly-plan.plan |   19 +
 ...btree-secondary-index-indexonly-plan-01.plan |   30 +
 ...btree-secondary-index-indexonly-plan-02.plan |   33 +
 ...btree-secondary-index-indexonly-plan-03.plan |   31 +
 ...btree-secondary-index-indexonly-plan-04.plan |   32 +
 ...btree-secondary-index-indexonly-plan-05.plan |   35 +
 ...btree-secondary-index-indexonly-plan-06.plan |   33 +
 ...btree-secondary-index-indexonly-plan-07.plan |   35 +
 ...btree-secondary-index-indexonly-plan-08.plan |   33 +
 ...btree-secondary-index-indexonly-plan-09.plan |   53 +
 ...pidx-with-join-rtree-sidx_01-index-only.plan |   41 +
 .../nonpure/keep-datetime-local-index-only.plan |   58 +
 ...rders-composite-index-search-index-only.plan |   30 +
 .../results/orders-index-search-index-only.plan |   29 +
 .../orders-index-search-open-index-only.plan    |   29 +
 .../rtree-index-join/issue730-index-only.plan   |   62 +
 ...pidx-with-join-rtree-sidx_01-index-only.plan |   62 +
 .../spatial-intersect-point_04.plan             |   47 +
 .../spatial-intersect-point_05.plan             |   47 +
 ...spatial-self-intersect-point-index-only.plan |   57 +
 ...ndexonly-plan-01-disable-indexonly-plan.plan |   22 +
 ...rtree-secondary-index-indexonly-plan-01.plan |   39 +
 ...rtree-secondary-index-indexonly-plan-02.plan |   33 +
 ...rtree-secondary-index-indexonly-plan-03.plan |   37 +
 ...rtree-secondary-index-indexonly-plan-04.plan |   38 +
 ...rtree-secondary-index-indexonly-plan-05.plan |   35 +
 ...rtree-secondary-index-indexonly-plan-06.plan |   39 +
 ...rtree-secondary-index-indexonly-plan-07.plan |   35 +
 ...rtree-secondary-index-indexonly-plan-08.plan |   33 +
 ...rtree-secondary-index-indexonly-plan-09.plan |   35 +
 .../rtree-secondary-index-index-only.plan       |   47 +
 .../rtree-secondary-index-open-index-only.plan  |   47 +
 .../udfs/query-ASTERIXDB-1019-index-only.plan   |  145 ++
 ...ad-with-secondary-btree-index-only.1.ddl.aql |   38 +
 ...with-secondary-btree-index-only.2.update.aql |   22 +
 ...ad-with-secondary-btree-index-only.3.ddl.aql |   21 +
 ...with-secondary-btree-index-only.4.update.aql |   18 +
 ...-with-secondary-btree-index-only.5.query.aql |   24 +
 ...ondary-indexonly-plan-equi-join_01.1.ddl.aql |   64 +
 ...ary-indexonly-plan-equi-join_01.2.update.aql |   36 +
 ...dary-indexonly-plan-equi-join_01.3.query.aql |   34 +
 .../btree-secondary-equi-join_04.1.ddl.aql      |   52 +
 .../btree-secondary-equi-join_04.2.update.aql   |   29 +
 .../btree-secondary-equi-join_04.3.ddl.aql      |   24 +
 .../btree-secondary-equi-join_04.4.query.aql    |   26 +
 ...xonly-plan-to-primary-equi-join_01.1.ddl.aql |   63 +
 ...ly-plan-to-primary-equi-join_01.2.update.aql |   29 +
 ...nly-plan-to-primary-equi-join_01.3.query.aql |   27 +
 ...ondary-indexonly-plan-equi-join_01.1.ddl.aql |   65 +
 ...ary-indexonly-plan-equi-join_01.2.update.aql |   29 +
 ...dary-indexonly-plan-equi-join_01.3.query.aql |   27 +
 ...xonly-plan-to-primary-equi-join_01.1.ddl.aql |   66 +
 ...ly-plan-to-primary-equi-join_01.2.update.aql |   29 +
 ...nly-plan-to-primary-equi-join_01.3.query.aql |   27 +
 ...ondary-indexonly-plan-equi-join_01.1.ddl.aql |   67 +
 ...ary-indexonly-plan-equi-join_01.2.update.aql |   29 +
 ...dary-indexonly-plan-equi-join_01.3.query.aql |   27 +
 .../rtree-spatial-intersect-point.1.ddl.aql     |   45 -
 .../rtree-spatial-intersect-point.2.update.aql  |   35 -
 .../rtree-spatial-intersect-point.3.ddl.aql     |   29 -
 .../rtree-spatial-intersect-point.4.query.aql   |   32 -
 .../rtree-spatial-intersect-point_01.1.ddl.aql  |   45 +
 ...tree-spatial-intersect-point_01.2.update.aql |   35 +
 .../rtree-spatial-intersect-point_01.3.ddl.aql  |   29 +
 ...rtree-spatial-intersect-point_01.4.query.aql |   32 +
 .../rtree-spatial-intersect-point_02.1.ddl.aql  |   46 +
 ...tree-spatial-intersect-point_02.2.update.aql |   29 +
 .../rtree-spatial-intersect-point_02.3.ddl.aql  |   30 +
 ...rtree-spatial-intersect-point_02.4.query.aql |   33 +
 .../rtree-spatial-intersect-point_03.1.ddl.aql  |   47 +
 ...tree-spatial-intersect-point_03.2.update.aql |   29 +
 .../rtree-spatial-intersect-point_03.3.ddl.aql  |   24 +
 ...rtree-spatial-intersect-point_03.4.query.aql |   26 +
 .../rtree-spatial-intersect-point_04.1.ddl.aql  |   47 +
 ...tree-spatial-intersect-point_04.2.update.aql |   29 +
 .../rtree-spatial-intersect-point_04.3.ddl.aql  |   24 +
 ...rtree-spatial-intersect-point_04.4.query.aql |   26 +
 .../rtree-spatial-intersect-point_05.1.ddl.aql  |   47 +
 ...tree-spatial-intersect-point_05.2.update.aql |   29 +
 .../rtree-spatial-intersect-point_05.3.ddl.aql  |   24 +
 ...rtree-spatial-intersect-point_05.4.query.aql |   26 +
 .../probe-pidx-with-join-btree-pidx1.1.ddl.aql  |   56 +
 ...robe-pidx-with-join-btree-pidx1.2.update.aql |   30 +
 ...probe-pidx-with-join-btree-pidx1.3.query.aql |   40 +
 ...robe-pidx-with-join-btree-sidx2.2.update.aql |    4 +-
 ...probe-pidx-with-join-btree-sidx2.3.query.aql |    6 +-
 ...th-join-btree-sidx3-indexonly-plan.1.ddl.aql |   56 +
 ...join-btree-sidx3-indexonly-plan.2.update.aql |   24 +
 ...-join-btree-sidx3-indexonly-plan.3.query.aql |   32 +
 ...dexonly-plan-with-join-btree-pidx1.1.ddl.aql |   57 +
 ...only-plan-with-join-btree-pidx1.2.update.aql |   24 +
 ...xonly-plan-with-join-btree-pidx1.3.query.aql |   32 +
 ...th-join-btree-sidx1-indexonly-plan.1.ddl.aql |   60 +
 ...join-btree-sidx1-indexonly-plan.2.update.aql |   24 +
 ...-join-btree-sidx1-indexonly-plan.3.query.aql |   32 +
 ...dexonly-plan-with-join-btree-pidx1.1.ddl.aql |   57 +
 ...only-plan-with-join-btree-pidx1.2.update.aql |   24 +
 ...xonly-plan-with-join-btree-pidx1.3.query.aql |   32 +
 ...th-join-btree-sidx1-indexonly-plan.1.ddl.aql |   59 +
 ...join-btree-sidx1-indexonly-plan.2.update.aql |   24 +
 ...-join-btree-sidx1-indexonly-plan.3.query.aql |   32 +
 ...-composite-index-indexonly-plan-01.1.ddl.aql |   62 +
 ...mposite-index-indexonly-plan-01.2.update.aql |   25 +
 ...omposite-index-indexonly-plan-01.3.query.aql |   25 +
 ...-composite-index-indexonly-plan-02.1.ddl.aql |   62 +
 ...mposite-index-indexonly-plan-02.2.update.aql |   24 +
 ...omposite-index-indexonly-plan-03.3.query.aql |   24 +
 ...-composite-index-indexonly-plan-03.1.ddl.aql |   62 +
 ...mposite-index-indexonly-plan-03.2.update.aql |   25 +
 ...omposite-index-indexonly-plan-03.3.query.aql |   24 +
 ...-secondary-index-indexonly-plan-01.1.ddl.aql |   60 +
 ...condary-index-indexonly-plan-01.10.query.aql |   28 +
 ...condary-index-indexonly-plan-01.2.update.aql |   25 +
 ...econdary-index-indexonly-plan-01.3.query.aql |   25 +
 ...econdary-index-indexonly-plan-01.4.query.aql |   25 +
 ...econdary-index-indexonly-plan-01.5.query.aql |   25 +
 ...econdary-index-indexonly-plan-01.6.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.7.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.8.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.9.query.aql |   27 +
 ...ondary-index-non-indexonly-plan-01.1.ddl.aql |   56 +
 ...ary-index-non-indexonly-plan-01.2.update.aql |   25 +
 ...dary-index-non-indexonly-plan-01.3.query.aql |   26 +
 ...-secondary-index-indexonly-plan-01.1.ddl.aql |  107 +
 ...condary-index-indexonly-plan-01.10.query.aql |   28 +
 ...condary-index-indexonly-plan-01.11.query.aql |   27 +
 ...condary-index-indexonly-plan-01.2.update.aql |   29 +
 ...econdary-index-indexonly-plan-01.3.query.aql |   24 +
 ...econdary-index-indexonly-plan-01.4.query.aql |   26 +
 ...econdary-index-indexonly-plan-01.5.query.aql |   26 +
 ...econdary-index-indexonly-plan-01.6.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.7.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.8.query.aql |   28 +
 ...econdary-index-indexonly-plan-01.9.query.aql |   28 +
 ...ondary-index-non-indexonly-plan-01.1.ddl.aql |   56 +
 ...ary-index-non-indexonly-plan-01.2.update.aql |   25 +
 ...dary-index-non-indexonly-plan-01.3.query.aql |   27 +
 ...ondary-index-non-indexonly-plan-02.1.ddl.aql |   57 +
 ...ary-index-non-indexonly-plan-02.2.update.aql |   25 +
 ...dary-index-non-indexonly-plan-02.3.query.aql |   28 +
 .../orderby_limit_02.3.query.aql                |    2 +-
 ...-with-secondary-btree-index-only.1.ddl.sqlpp |   40 +
 ...th-secondary-btree-index-only.2.update.sqlpp |   24 +
 ...-with-secondary-btree-index-only.3.ddl.sqlpp |   24 +
 ...th-secondary-btree-index-only.4.update.sqlpp |   19 +
 ...ith-secondary-btree-index-only.5.query.sqlpp |   26 +
 ...dary-indexonly-plan-equi-join_01.1.ddl.sqlpp |   64 +
 ...y-indexonly-plan-equi-join_01.2.update.sqlpp |   36 +
 ...ry-indexonly-plan-equi-join_01.3.query.sqlpp |   32 +
 .../btree-secondary-equi-join_04.1.ddl.sqlpp    |   52 +
 .../btree-secondary-equi-join_04.2.update.sqlpp |   29 +
 .../btree-secondary-equi-join_04.3.ddl.sqlpp    |   24 +
 .../btree-secondary-equi-join_04.4.query.sqlpp  |   25 +
 ...nly-plan-to-primary-equi-join_01.1.ddl.sqlpp |   63 +
 ...-plan-to-primary-equi-join_01.2.update.sqlpp |   29 +
 ...y-plan-to-primary-equi-join_01.3.query.sqlpp |   26 +
 ...dary-indexonly-plan-equi-join_01.1.ddl.sqlpp |   65 +
 ...y-indexonly-plan-equi-join_01.2.update.sqlpp |   29 +
 ...ry-indexonly-plan-equi-join_01.3.query.sqlpp |   26 +
 ...nly-plan-to-primary-equi-join_01.1.ddl.sqlpp |   66 +
 ...-plan-to-primary-equi-join_01.2.update.sqlpp |   29 +
 ...y-plan-to-primary-equi-join_01.3.query.sqlpp |   26 +
 ...dary-indexonly-plan-equi-join_01.1.ddl.sqlpp |   67 +
 ...y-indexonly-plan-equi-join_01.2.update.sqlpp |   29 +
 ...ry-indexonly-plan-equi-join_01.3.query.sqlpp |   26 +
 .../rtree-spatial-intersect-point.1.ddl.sqlpp   |   48 -
 ...rtree-spatial-intersect-point.2.update.sqlpp |   32 -
 .../rtree-spatial-intersect-point.3.ddl.sqlpp   |   30 -
 .../rtree-spatial-intersect-point.4.query.sqlpp |   34 -
 ...rtree-spatial-intersect-point_01.1.ddl.sqlpp |   48 +
 ...ee-spatial-intersect-point_01.2.update.sqlpp |   32 +
 ...rtree-spatial-intersect-point_01.3.ddl.sqlpp |   30 +
 ...ree-spatial-intersect-point_01.4.query.sqlpp |   34 +
 ...rtree-spatial-intersect-point_02.1.ddl.sqlpp |   46 +
 ...ee-spatial-intersect-point_02.2.update.sqlpp |   29 +
 ...rtree-spatial-intersect-point_02.3.ddl.sqlpp |   30 +
 ...ree-spatial-intersect-point_02.4.query.sqlpp |   34 +
 ...rtree-spatial-intersect-point_03.1.ddl.sqlpp |   47 +
 ...ee-spatial-intersect-point_03.2.update.sqlpp |   29 +
 ...rtree-spatial-intersect-point_03.3.ddl.sqlpp |   24 +
 ...ree-spatial-intersect-point_03.4.query.sqlpp |   27 +
 ...rtree-spatial-intersect-point_04.1.ddl.sqlpp |   47 +
 ...ee-spatial-intersect-point_04.2.update.sqlpp |   29 +
 ...rtree-spatial-intersect-point_04.3.ddl.sqlpp |   24 +
 ...ree-spatial-intersect-point_04.4.query.sqlpp |   27 +
 ...rtree-spatial-intersect-point_05.1.ddl.sqlpp |   47 +
 ...ee-spatial-intersect-point_05.2.update.sqlpp |   29 +
 ...rtree-spatial-intersect-point_05.3.ddl.sqlpp |   24 +
 ...ree-spatial-intersect-point_05.4.query.sqlpp |   27 +
 ...-join-btree-sidx3-indexonly-plan.1.ddl.sqlpp |   56 +
 ...in-btree-sidx3-indexonly-plan.2.update.sqlpp |   24 +
 ...oin-btree-sidx3-indexonly-plan.3.query.sqlpp |   29 +
 ...xonly-plan-with-join-btree-pidx1.1.ddl.sqlpp |   57 +
 ...ly-plan-with-join-btree-pidx1.2.update.sqlpp |   24 +
 ...nly-plan-with-join-btree-pidx1.3.query.sqlpp |   30 +
 ...-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp |   60 +
 ...in-btree-sidx1-indexonly-plan.2.update.sqlpp |   24 +
 ...oin-btree-sidx1-indexonly-plan.3.query.sqlpp |   29 +
 ...xonly-plan-with-join-btree-pidx1.1.ddl.sqlpp |   57 +
 ...ly-plan-with-join-btree-pidx1.2.update.sqlpp |   24 +
 ...nly-plan-with-join-btree-pidx1.3.query.sqlpp |   30 +
 ...-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp |   59 +
 ...in-btree-sidx1-indexonly-plan.2.update.sqlpp |   24 +
 ...oin-btree-sidx1-indexonly-plan.3.query.sqlpp |   29 +
 ...omposite-index-indexonly-plan-01.1.ddl.sqlpp |   60 +
 ...osite-index-indexonly-plan-01.2.update.sqlpp |   25 +
 ...posite-index-indexonly-plan-01.3.query.sqlpp |   25 +
 ...omposite-index-indexonly-plan-02.1.ddl.sqlpp |   60 +
 ...osite-index-indexonly-plan-02.2.update.sqlpp |   24 +
 ...posite-index-indexonly-plan-02.3.query.sqlpp |   24 +
 ...omposite-index-indexonly-plan-03.1.ddl.sqlpp |   60 +
 ...osite-index-indexonly-plan-03.2.update.sqlpp |   25 +
 ...posite-index-indexonly-plan-03.3.query.sqlpp |   25 +
 ...econdary-index-indexonly-plan-01.1.ddl.sqlpp |   60 +
 ...ndary-index-indexonly-plan-01.10.query.sqlpp |   27 +
 ...ndary-index-indexonly-plan-01.2.update.sqlpp |   25 +
 ...ondary-index-indexonly-plan-01.3.query.sqlpp |   25 +
 ...ondary-index-indexonly-plan-01.4.query.sqlpp |   25 +
 ...ondary-index-indexonly-plan-01.5.query.sqlpp |   25 +
 ...ondary-index-indexonly-plan-01.6.query.sqlpp |   26 +
 ...ondary-index-indexonly-plan-01.7.query.sqlpp |   27 +
 ...ondary-index-indexonly-plan-01.8.query.sqlpp |   26 +
 ...ondary-index-indexonly-plan-01.9.query.sqlpp |   26 +
 ...dary-index-non-indexonly-plan-01.1.ddl.sqlpp |   56 +
 ...y-index-non-indexonly-plan-01.2.update.sqlpp |   25 +
 ...ry-index-non-indexonly-plan-01.3.query.sqlpp |   25 +
 ...econdary-index-indexonly-plan-01.1.ddl.sqlpp |  107 +
 ...ndary-index-indexonly-plan-01.10.query.sqlpp |   27 +
 ...ndary-index-indexonly-plan-01.11.query.sqlpp |   26 +
 ...ndary-index-indexonly-plan-01.2.update.sqlpp |   29 +
 ...ondary-index-indexonly-plan-01.3.query.sqlpp |   26 +
 ...ondary-index-indexonly-plan-01.4.query.sqlpp |   26 +
 ...ondary-index-indexonly-plan-01.5.query.sqlpp |   26 +
 ...ondary-index-indexonly-plan-01.6.query.sqlpp |   27 +
 ...ondary-index-indexonly-plan-01.7.query.sqlpp |   27 +
 ...ondary-index-indexonly-plan-01.8.query.sqlpp |   27 +
 ...ondary-index-indexonly-plan-01.9.query.sqlpp |   27 +
 ...dary-index-non-indexonly-plan-01.1.ddl.sqlpp |   56 +
 ...y-index-non-indexonly-plan-01.2.update.sqlpp |   25 +
 ...ry-index-non-indexonly-plan-01.3.query.sqlpp |   27 +
 ...dary-index-non-indexonly-plan-02.1.ddl.sqlpp |   57 +
 ...y-index-non-indexonly-plan-02.2.update.sqlpp |   25 +
 ...ry-index-non-indexonly-plan-02.3.query.sqlpp |   27 +
 .../load-with-secondary-btree-index-only.1.adm  |   15 +
 ...-secondary-indexonly-plan-equi-join_01.1.adm |    1 +
 .../btree-secondary-equi-join_04.1.adm          |    5 +
 ...indexonly-plan-to-primary-equi-join_01.1.adm |    1 +
 ...-secondary-indexonly-plan-equi-join_01.1.adm |    1 +
 ...indexonly-plan-to-primary-equi-join_01.1.adm |    1 +
 ...-secondary-indexonly-plan-equi-join_01.1.adm |    1 +
 .../rtree-spatial-intersect-point.1.adm         |   44 -
 .../rtree-spatial-intersect-point_01.1.adm      |   44 +
 .../rtree-spatial-intersect-point_02.1.adm      |   65 +
 .../rtree-spatial-intersect-point_03.1.adm      |   65 +
 .../rtree-spatial-intersect-point_04.1.adm      |   20 +
 .../rtree-spatial-intersect-point_05.1.adm      |  204 ++
 .../probe-pidx-with-join-btree-pidx1.1.adm      |    1 +
 ...x-with-join-rtree-sidx3-indexonly-plan.1.adm |    1 +
 ...e-indexonly-plan-with-join-btree-pidx1.1.adm |    1 +
 ...n-with-join-btree-sidx1-indexonly-plan.1.adm |    1 +
 ...n-indexonly-plan-with-join-btree-pidx1.1.adm |    1 +
 ...n-with-join-btree-sidx1-indexonly-plan.1.adm |    1 +
 ...dary-composite-index-indexonly-plan-01.1.adm |    2 +
 ...dary-composite-index-indexonly-plan-02.1.adm |    2 +
 ...dary-composite-index-indexonly-plan-03.1.adm |    2 +
 ...ree-secondary-index-indexonly-plan-01.10.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.3.adm |    2 +
 ...tree-secondary-index-indexonly-plan-01.4.adm |    2 +
 ...tree-secondary-index-indexonly-plan-01.5.adm |    2 +
 ...tree-secondary-index-indexonly-plan-01.6.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.7.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.8.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.9.adm |    1 +
 ...-secondary-index-non-indexonly-plan-01.1.adm |    1 +
 ...ree-secondary-index-indexonly-plan-01.10.adm |    1 +
 ...ree-secondary-index-indexonly-plan-01.11.adm |   30 +
 ...tree-secondary-index-indexonly-plan-01.3.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.4.adm |    6 +
 ...tree-secondary-index-indexonly-plan-01.5.adm |    6 +
 ...tree-secondary-index-indexonly-plan-01.6.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.7.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.8.adm |    1 +
 ...tree-secondary-index-indexonly-plan-01.9.adm |    1 +
 ...-secondary-index-non-indexonly-plan-01.1.adm |    1 +
 ...-secondary-index-non-indexonly-plan-02.1.adm |    1 +
 .../orderby_limit_02/orderby_limit_02.1.adm     |    4 +-
 .../src/test/resources/runtimets/testsuite.xml  |  124 +-
 .../resources/runtimets/testsuite_sqlpp.xml     |  126 +-
 .../asterix/common/exceptions/ErrorCode.java    |    6 +
 .../main/resources/asx_errormsg/en.properties   |    6 +
 .../markdown/sqlpp/appendix_2_parameters.md     |   18 +
 .../metadata/declared/DatasetDataSource.java    |    2 +-
 .../metadata/declared/MetadataProvider.java     |   60 +-
 .../asterix/metadata/entities/Dataset.java      |   38 +-
 .../serde/SerializerDeserializerUtil.java       |   21 +
 .../om/types/hierachy/ATypeHierarchy.java       |   54 +-
 .../DoubleToFloatTypeConvertComputer.java       |    8 +-
 .../DoubleToInt16TypeConvertComputer.java       |    7 +-
 .../DoubleToInt32TypeConvertComputer.java       |    7 +-
 .../DoubleToInt64TypeConvertComputer.java       |    7 +-
 .../DoubleToInt8TypeConvertComputer.java        |    7 +-
 .../FloatToDoubleTypeConvertComputer.java       |   10 +-
 .../FloatToInt16TypeConvertComputer.java        |    7 +-
 .../FloatToInt32TypeConvertComputer.java        |    7 +-
 .../FloatToInt64TypeConvertComputer.java        |    7 +-
 .../FloatToInt8TypeConvertComputer.java         |    7 +-
 .../om/types/hierachy/ITypeConvertComputer.java |    4 +-
 .../IntegerToDoubleTypeConvertComputer.java     |    4 +-
 .../IntegerToFloatTypeConvertComputer.java      |    4 +-
 .../IntegerToInt16TypeConvertComputer.java      |    4 +-
 .../IntegerToInt32TypeConvertComputer.java      |    4 +-
 .../IntegerToInt64TypeConvertComputer.java      |    4 +-
 .../IntegerToInt8TypeConvertComputer.java       |    4 +-
 ...xInstanctSearchOperationCallbackFactory.java |   68 +
 ...daryIndexInstantSearchOperationCallback.java |   72 +
 .../algebricks/common/utils/Quadruple.java      |   84 +
 .../logical/AbstractUnnestMapOperator.java      |   27 +-
 .../logical/LeftOuterUnnestMapOperator.java     |    3 +-
 .../LogicalOperatorPrettyPrintVisitor.java      |    2 +-
 .../LogicalOperatorPrettyPrintVisitorJson.java  |    3 +-
 .../algebra/util/OperatorPropertiesUtil.java    |   20 +-
 .../rules/ExtractCommonOperatorsRule.java       |    3 +-
 .../rewriter/rules/IntroduceProjectsRule.java   |   33 +-
 .../RemoveUnusedAssignAndAggregateRule.java     |    5 +-
 .../hyracks/api/exceptions/ErrorCode.java       |    2 +
 .../src/main/resources/errormsg/en.properties   |    2 +
 .../dataflow/BTreeSearchOperatorDescriptor.java |   27 +-
 .../BTreeSearchOperatorNodePushable.java        |   16 +-
 .../storage/am/common/api/ILSMIndexCursor.java  |   10 +
 .../IndexSearchOperatorNodePushable.java        |   44 +-
 .../btree/impls/LSMBTreePointSearchCursor.java  |   11 +-
 .../btree/impls/LSMBTreeRangeSearchCursor.java  |   88 +-
 .../lsm/btree/impls/LSMBTreeSearchCursor.java   |    5 +
 .../impls/LSMBTreeWithBuddyAbstractCursor.java  |    6 +
 .../lsm/common/impls/LSMIndexSearchCursor.java  |    5 +
 .../impls/LSMInvertedIndexSearchCursor.java     |    6 +
 .../lsm/rtree/impls/LSMRTreeAbstractCursor.java |   10 +-
 .../lsm/rtree/impls/LSMRTreeSearchCursor.java   |    8 +
 ...LSMRTreeWithAntiMatterTuplesFlushCursor.java |    5 +
 ...SMRTreeWithAntiMatterTuplesSearchCursor.java |   12 +-
 .../dataflow/RTreeSearchOperatorDescriptor.java |   20 +-
 .../RTreeSearchOperatorNodePushable.java        |   16 +-
 425 files changed, 16222 insertions(+), 1023 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
----------------------------------------------------------------------
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 11b86cb..beb47a8 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
@@ -125,7 +125,8 @@ public class BTreeSearchPOperator extends IndexSearchPOperator {
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
                 jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
-                jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
+                jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
+                unnestMap.getGenerateCallBackProceedResultVar());
 
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 733e62f..949be23 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -98,10 +98,10 @@ public class RTreeSearchPOperator extends IndexSearchPOperator {
             // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
             retainNull = true;
         }
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch =
-                mp.buildRtreeRuntime(builder.getJobSpec(), outputVars, opSchema, typeEnv, context,
-                        jobGenParams.getRetainInput(), retainNull, dataset, jobGenParams.getIndexName(), keyIndexes,
-                        propagateIndexFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
+                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
+                dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
+                maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
 
         builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index bd67ed4..cd217ab 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -332,7 +332,6 @@ public final class RuleCollections {
         physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
         physicalRewritesAllLevels.add(new RemoveUnnecessarySortMergeExchange());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
-        physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
         physicalRewritesAllLevels.add(new IntroduceMaterializationForInsertWithSelfScanRule());
         physicalRewritesAllLevels.add(new InlineSingleReferenceVariablesRule());
         physicalRewritesAllLevels.add(new RemoveUnusedAssignAndAggregateRule());
@@ -361,6 +360,8 @@ public final class RuleCollections {
 
     public static final List<IAlgebraicRewriteRule> prepareForJobGenRuleCollection() {
         List<IAlgebraicRewriteRule> prepareForJobGenRewrites = new LinkedList<>();
+        prepareForJobGenRewrites.add(new InsertProjectBeforeUnionRule());
+        prepareForJobGenRewrites.add(new SetAlgebricksPhysicalOperatorsRule());
         prepareForJobGenRewrites
                 .add(new IsolateHyracksOperatorsRule(HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index e0c6249..f4178d7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -44,6 +45,7 @@ import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,6 +65,9 @@ import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFun
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
@@ -75,6 +80,9 @@ import com.google.common.collect.ImmutableSet;
  * methods.
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
+    // When this option is set to true before executing a query, we don't apply the index-only plan.
+    public final static String NO_INDEX_ONLY_PLAN_OPTION = "noindexonly";
+    public final static boolean NO_INDEX_ONLY_PLAN_OPTION_DEFAULT_VALUE = false;
 
     protected MetadataProvider metadataProvider;
 
@@ -91,12 +99,12 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
 
     protected static void registerAccessMethod(IAccessMethod accessMethod,
             Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
-        List<FunctionIdentifier> funcs = accessMethod.getOptimizableFunctions();
-        for (FunctionIdentifier funcIdent : funcs) {
-            List<IAccessMethod> l = accessMethods.get(funcIdent);
+        List<Pair<FunctionIdentifier, Boolean>> funcs = accessMethod.getOptimizableFunctions();
+        for (Pair<FunctionIdentifier, Boolean> funcIdent : funcs) {
+            List<IAccessMethod> l = accessMethods.get(funcIdent.first);
             if (l == null) {
-                l = new ArrayList<>();
-                accessMethods.put(funcIdent, l);
+                l = new ArrayList<IAccessMethod>();
+                accessMethods.put(funcIdent.first, l);
             }
             l.add(accessMethod);
         }
@@ -550,16 +558,13 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         int optFuncExprIndex = 0;
         List<Index> datasetIndexes = new ArrayList<>();
         LogicalVariable datasetMetaVar = null;
-        LogicalVariable datasetRecordVar = null;
-        if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
+        if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN
+                && subTree.getDataSourceType() != DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP) {
             datasetIndexes = metadataProvider.getDatasetIndexes(subTree.getDataset().getDataverseName(),
                     subTree.getDataset().getDatasetName());
             List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
             if (subTree.getDataset().hasMetaPart()) {
                 datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
-                datasetRecordVar = datasetVars.get(datasetVars.size() - 2);
-            } else {
-                datasetRecordVar = datasetVars.get(datasetVars.size() - 1);
             }
         }
         for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
@@ -568,11 +573,11 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     .size(); assignOrUnnestIndex++) {
                 AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
                 if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-                    analyzeAssignOp((AssignOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetRecordVar,
-                            datasetMetaVar, context, datasetIndexes, optFuncExprIndex, analysisCtx);
+                    analyzeAssignOp((AssignOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetMetaVar,
+                            context, datasetIndexes, optFuncExprIndex, analysisCtx);
                 } else {
-                    analyzeUnnestOp((UnnestOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetRecordVar,
-                            datasetMetaVar, context, datasetIndexes, optFuncExprIndex, analysisCtx);
+                    analyzeUnnestOp((UnnestOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetMetaVar,
+                            context, datasetIndexes, optFuncExprIndex, analysisCtx);
                 }
             }
 
@@ -602,9 +607,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     }
 
     private void analyzeUnnestOp(UnnestOperator unnestOp, IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetRecordVar,
-            LogicalVariable datasetMetaVar, IOptimizationContext context, List<Index> datasetIndexes,
-            int optFuncExprIndex, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+            OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
+            IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         LogicalVariable var = unnestOp.getVariable();
         int funcVarIndex = optFuncExpr.findLogicalVar(var);
         // No matching var in optFuncExpr.
@@ -621,7 +626,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         } else {
             fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
                     funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
-                    datasetRecordVar, subTree.getMetaRecordType(), datasetMetaVar);
+                    subTree.getMetaRecordType(), datasetMetaVar);
             if (fieldName.isEmpty()) {
                 return;
             }
@@ -641,9 +646,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     }
 
     private void analyzeAssignOp(AssignOperator assignOp, IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetRecordVar,
-            LogicalVariable datasetMetaVar, IOptimizationContext context, List<Index> datasetIndexes,
-            int optFuncExprIndex, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+            OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
+            IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         List<LogicalVariable> varList = assignOp.getVariables();
         for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
             LogicalVariable var = varList.get(varIndex);
@@ -659,8 +664,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
 
             List<String> fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
                     subTree.getRecordType(), optVarIndex,
-                    optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), datasetRecordVar,
-                    subTree.getMetaRecordType(), datasetMetaVar);
+                    optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), subTree.getMetaRecordType(),
+                    datasetMetaVar);
 
             IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
             // Set the fieldName in the corresponding matched
@@ -742,14 +747,14 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
 
     /**
      * Returns the field name corresponding to the assigned variable at
-     * varIndex. Returns null if the expr at varIndex does not yield to a field
+     * varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
      * access function after following a set of allowed functions.
      *
      * @throws AlgebricksException
      */
     protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
             int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
-            ILogicalExpression parentFuncExpr, LogicalVariable recordVar, ARecordType metaType, LogicalVariable metaVar)
+            ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar)
             throws AlgebricksException {
         // Get expression corresponding to opVar at varIndex.
         AbstractLogicalExpression expr = null;
@@ -818,7 +823,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         if (isFieldAccess) {
             LogicalVariable sourceVar =
                     ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
-            optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
+            if (optFuncExpr != null) {
+                optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
+            }
             int[] assignAndExpressionIndexes = null;
 
             //go forward through nested assigns until you find the relevant one
@@ -856,7 +863,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 //Recursive call on nested assign
                 List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
                         assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
-                        parentFuncExpr, recordVar, metaType, metaVar);
+                        parentFuncExpr, metaType, metaVar);
 
                 if (parentFieldNames.isEmpty()) {
                     //Nested assign was not a field access.
@@ -869,7 +876,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                             ? ((ARecordType) metaType.getSubFieldType(parentFieldNames)).getFieldNames()[fieldIndex]
                             : ((ARecordType) recordType.getSubFieldType(parentFieldNames)).getFieldNames()[fieldIndex];
                 }
-                optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+                if (optFuncExpr != null) {
+                    optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+                }
                 //add fieldName to the nested fieldName, return
                 if (nestedAccessFieldName != null) {
                     for (int i = 0; i < nestedAccessFieldName.size(); i++) {
@@ -881,7 +890,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 return (parentFieldNames);
             }
 
-            optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+            if (optFuncExpr != null) {
+                optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+            }
             //no nested assign, we are at the lowest level.
             if (isByName) {
                 if (nestedAccessFieldName != null) {
@@ -898,7 +909,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             return Collections.emptyList();
         }
         // We use a part of the field in edit distance computation
-        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
+        if (optFuncExpr != null
+                && optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
             optFuncExpr.setPartialField(true);
         }
         // We expect the function's argument to be a variable, otherwise we
@@ -918,10 +930,10 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 List<LogicalVariable> varList = assignOp.getVariables();
                 for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
                     LogicalVariable var = varList.get(varIndex);
-                    if (var.equals(curVar)) {
+                    if (var.equals(curVar) && optFuncExpr != null) {
                         optFuncExpr.setSourceVar(funcVarIndex, var);
                         return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
-                                funcVarIndex, childFuncExpr, recordVar, metaType, metaVar);
+                                funcVarIndex, childFuncExpr, metaType, metaVar);
                     }
                 }
             } else {
@@ -929,10 +941,115 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 LogicalVariable var = unnestOp.getVariable();
                 if (var.equals(curVar)) {
                     getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
-                            childFuncExpr, recordVar, metaType, metaVar);
+                            childFuncExpr, metaType, metaVar);
                 }
             }
         }
         return Collections.emptyList();
     }
+
+    /**
+     * Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
+     */
+    protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
+        LogicalVariable datasetMetaVar = null;
+        if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN
+                && subTree.getDataSourceType() != DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP) {
+            List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
+            if (subTree.getDataset().hasMetaPart()) {
+                datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
+            }
+        }
+        for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
+                .size(); assignOrUnnestIndex++) {
+            AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
+            if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                AssignOperator assignOp = (AssignOperator) op;
+                List<LogicalVariable> varList = assignOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    // funcVarIndex is not required. Thus, we set it to -1.
+                    // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
+                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                    if (fieldName != null && !fieldName.isEmpty()) {
+                        subTree.getVarsToFieldNameMap().put(var, fieldName);
+                    }
+                }
+            } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+                UnnestOperator unnestOp = (UnnestOperator) op;
+                LogicalVariable var = unnestOp.getVariable();
+                List<String> fieldName = null;
+                if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
+                    // funcVarIndex is not required. Thus, we set it to -1.
+                    // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
+                            -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                    if (fieldName != null && !fieldName.isEmpty()) {
+                        subTree.getVarsToFieldNameMap().put(var, fieldName);
+                    }
+                }
+            } else {
+                // unnestmap or left-outer-unnestmap?
+                LeftOuterUnnestMapOperator leftOuterUnnestMapOp = null;
+                UnnestMapOperator unnestMapOp = null;
+                List<LogicalVariable> varList = null;
+
+                if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                    unnestMapOp = (UnnestMapOperator) op;
+                    varList = unnestMapOp.getVariables();
+                } else if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
+                    leftOuterUnnestMapOp = (LeftOuterUnnestMapOperator) op;
+                    varList = leftOuterUnnestMapOp.getVariables();
+                } else {
+                    continue;
+                }
+
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    // funcVarIndex is not required. Thus, we set it to -1.
+                    // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
+                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                    if (fieldName != null && !fieldName.isEmpty()) {
+                        subTree.getVarsToFieldNameMap().put(var, fieldName);
+                    }
+                }
+            }
+        }
+
+        // DatasourceScan?
+        if (subTree.hasDataSourceScan()) {
+            List<LogicalVariable> primaryKeyVarList = new ArrayList<>();
+
+            if (subTree.getDataset().getDatasetType() == DatasetType.INTERNAL) {
+                subTree.getPrimaryKeyVars(null, primaryKeyVarList);
+
+                Index primaryIndex = getPrimaryIndexFromDataSourceScanOp(subTree.getDataSourceRef().getValue());
+
+                for (int i = 0; i < primaryKeyVarList.size(); i++) {
+                    subTree.getVarsToFieldNameMap().put(primaryKeyVarList.get(i),
+                            primaryIndex.getKeyFieldNames().get(i));
+                }
+            }
+
+        }
+
+    }
+
+    /**
+     * Fetches the associated primary index from the given DATASOURCESCAN operator.
+     */
+    protected Index getPrimaryIndexFromDataSourceScanOp(ILogicalOperator dataSourceScanOp) throws AlgebricksException {
+        if (dataSourceScanOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+            return null;
+        }
+        Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo((DataSourceScanOperator) dataSourceScanOp);
+        String dataverseName = datasetInfo.first;
+        String datasetName = datasetInfo.second;
+
+        Index idxUsedInUnnestMap = metadataProvider.getIndex(dataverseName, datasetName, datasetName);
+        return idxUsedInUnnestMap;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index 16ee6d1..c186687 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -28,6 +28,7 @@ import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Quadruple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 
@@ -52,7 +53,40 @@ public class AccessMethodAnalysisContext {
 
     // variables for resetting null placeholder for left-outer-join
     private Mutable<ILogicalOperator> lojGroupbyOpRef = null;
-    private ScalarFunctionCallExpression lojIsNullFuncInGroupBy = null;
+    private ScalarFunctionCallExpression lojIsMissingFuncInGroupBy = null;
+
+    // For a secondary index, if we use only PK and secondary key field in a plan, it is an index-only plan.
+    // Contains information about index-only plan
+    //
+    // 1. isIndexOnlyPlan - index-only plan possible?
+    //    This option is the primary option. If this is false, then regardless of the following variables,
+    //    An index-only plan will not be constructed. If this is true, then we use the following variables to
+    //    construct an index-only plan.
+    //
+    // 2. secondaryKeyFieldUsedAfterSelectOrJoinOp - secondary key field usage after the select or join operator?
+    //    If the secondary key field is used after SELECT or JOIN operator (e.g., returning the field),
+    //    then we need to keep secondary keys from the secondary index search.
+    //
+    // 3. requireVerificationAfterSIdxSearch -
+    //    whether a verification (especially for R-Tree case) is required after the secondary index search?
+    //    For an R-Tree index, if the given query shape is not RECTANGLE or POINT,
+    //    we need to add the original SELECT operator to filter out the false positive results.
+    //    (e.g., spatial-intersect($o.pointfield, create-circle(create-point(30.0,70.0), 5.0)) )
+    //
+    //    Also, for a B-Tree composite index, we need to apply SELECT operators in the right path
+    //    to remove any false positive results from the secondary composite index search.
+    //
+    //    Lastly, if there is an index-nested-loop-join and the join contains more conditions
+    //    other than joining fields, then those conditions need to be applied to filter out
+    //    false positive results in the right path (isntantTryLock success path).
+    //    (e.g., where $a.authors /*+ indexnl */ = $b.authors and $a.id = $b.id)
+    //    For more details, refer to AccessMethodUtils.createPrimaryIndexUnnestMap() method.
+    //
+    // 4. doesSIdxSearchCoverAllPredicates - can the given index cover all search predicates?
+    //    In other words, all search predicates are about the given secondary index?
+    //
+    private Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo =
+            new Quadruple<>(false, false, false, false);
 
     public void addIndexExpr(Dataset dataset, Index index, Integer exprIndex, Integer varIndex) {
         List<Pair<Integer, Integer>> exprs = getIndexExprsFromIndexExprsAndVars(index);
@@ -108,20 +142,36 @@ public class AccessMethodAnalysisContext {
         return lojGroupbyOpRef;
     }
 
-    public void setLOJIsNullFuncInGroupBy(ScalarFunctionCallExpression isNullFunc) {
-        lojIsNullFuncInGroupBy = isNullFunc;
+    public void setLOJIsMissingFuncInGroupBy(ScalarFunctionCallExpression isMissingFunc) {
+        lojIsMissingFuncInGroupBy = isMissingFunc;
     }
 
-    public ScalarFunctionCallExpression getLOJIsNullFuncInGroupBy() {
-        return lojIsNullFuncInGroupBy;
+    public ScalarFunctionCallExpression getLOJIsMissingFuncInGroupBy() {
+        return lojIsMissingFuncInGroupBy;
     }
 
     public Dataset getDatasetFromIndexDatasetMap(Index idx) {
-        return indexDatasetMap.get(idx);
+        return getIndexDatasetMap().get(idx);
     }
 
     public void putDatasetIntoIndexDatasetMap(Index idx, Dataset ds) {
-        indexDatasetMap.put(idx, ds);
+        getIndexDatasetMap().put(idx, ds);
+    }
+
+    public void setIndexOnlyPlanInfo(Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo) {
+        this.indexOnlyPlanInfo = indexOnlyPlanInfo;
+    }
+
+    public Quadruple<Boolean, Boolean, Boolean, Boolean> getIndexOnlyPlanInfo() {
+        return this.indexOnlyPlanInfo;
+    }
+
+    public Map<Index, Dataset> getIndexDatasetMap() {
+        return indexDatasetMap;
+    }
+
+    public void setIndexDatasetMap(Map<Index, Dataset> indexDatasetMap) {
+        this.indexDatasetMap = indexDatasetMap;
     }
 
 }


[07/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.aql
new file mode 100644
index 0000000..11a3605
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.aql
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+//create index btree_index_docid on MyData(docid) type btree;
+//create index btree_index_val1 on MyData(val1) type btree;
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.aql
new file mode 100644
index 0000000..07ad6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-03.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
new file mode 100644
index 0000000..6993c15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id, "sk":$o.docid}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.aql
new file mode 100644
index 0000000..7b436d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.aql
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select   -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split -> select (the second condition) -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+//create index btree_index_docid on MyData(docid) type btree;
+//create index btree_index_val1 on MyData(val1) type btree;
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.aql
new file mode 100644
index 0000000..4f5e20d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
new file mode 100644
index 0000000..256baa8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3 and $o.val1 >= 3
+return {"pk":$o.id, "sk":$o.docid, "sk2":$o.val1}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.aql
new file mode 100644
index 0000000..5935213
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.aql
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.aql
new file mode 100644
index 0000000..72bf03e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return $o
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.aql
new file mode 100644
index 0000000..9a7b2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.aql
new file mode 100644
index 0000000..2e3b770
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id, "sk":$o.docid}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.aql
new file mode 100644
index 0000000..eb7d8d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.aql
new file mode 100644
index 0000000..019ec61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"sk":$o.docid}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.aql
new file mode 100644
index 0000000..dbfceea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id, "sk":$o.docid}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.aql
new file mode 100644
index 0000000..0eaa1ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.aql
new file mode 100644
index 0000000..d42794f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"sk":$o.docid}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.aql
new file mode 100644
index 0000000..50fd514
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"rec":$o, "pk":$o.id}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.aql
new file mode 100644
index 0000000..a6b865b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.aql
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we have multiple conditions that one index can cover.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.aql
new file mode 100644
index 0000000..9a7b2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.aql
new file mode 100644
index 0000000..d072af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 10 and $o.val1 < 3
+return {"pk":$o.id, "sk":$o.point}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.aql
new file mode 100644
index 0000000..462122a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.aql
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.aql
new file mode 100644
index 0000000..98d86bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return $o
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.aql
new file mode 100644
index 0000000..577c6f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+//let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+let $area:=create-circle(create-point(4.0,3.0), 5.0)
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.aql
new file mode 100644
index 0000000..cf3f316
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+
+load dataset Fragile_raw using localfs
+(("path"="asterix_nc1://data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.aql
new file mode 100644
index 0000000..9f7e04a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count (for $x in dataset Fragile_raw where
+spatial-intersect($x.location, create-polygon([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
+return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.aql
new file mode 100644
index 0000000..c293790
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.aql
new file mode 100644
index 0000000..6a20f70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"point":$o.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.aql
new file mode 100644
index 0000000..9e7718b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id, "point":$o.point}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.aql
new file mode 100644
index 0000000..c8ad5fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.aql
new file mode 100644
index 0000000..371aa4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"point":$o.point}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.aql
new file mode 100644
index 0000000..ca7658b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"rec":$o, "id":$o.id}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.aql
new file mode 100644
index 0000000..b360ab6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.aql
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary full-text index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we have multiple conditions that one index can cover.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.aql
new file mode 100644
index 0000000..4f5e20d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.aql
new file mode 100644
index 0000000..4d1079c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area) and $o.val1 < 30
+return {"id":$o.id, "point":$o.point}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.aql
new file mode 100644
index 0000000..a1d2872
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.aql
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary full-text index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we have a condition that uses a R-Tree index on polygon fields.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index rtree_index_polygon on MyData(poly1) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.aql
new file mode 100644
index 0000000..4f5e20d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.aql
new file mode 100644
index 0000000..66bf653
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+let $ps := [point("1.0,1.0"), point("3.0,3.0")]
+for $p in $ps
+for $o in dataset('MyData')
+where spatial-intersect($p, $o.poly1)
+return {"id":$o.id, "point":$o.point}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/orderby_limit/orderby_limit_02/orderby_limit_02.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/orderby_limit/orderby_limit_02/orderby_limit_02.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/orderby_limit/orderby_limit_02/orderby_limit_02.3.query.aql
index 93288aa..575b8a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/orderby_limit/orderby_limit_02/orderby_limit_02.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/orderby_limit/orderby_limit_02/orderby_limit_02.3.query.aql
@@ -28,6 +28,6 @@ use dataverse test;
 
 for $l in dataset Employee
 where $l.name >= "A" and $l.name <= "Z"
-limit 5
 order by $l.name desc
+limit 5
 return {"name": $l.name}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.sqlpp
new file mode 100644
index 0000000..2acbf70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test filters with loading and in the existence of a secondary b-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type test.FacebookMessageType as
+ closed {
+  `message-id` : bigint,
+  `author-id` : bigint,
+  `in-response-to` : bigint?,
+  `sender-location` : point?,
+  message : string,
+  `send-time` : datetime
+};
+
+create  dataset FacebookMessages(FacebookMessageType) primary key `message-id` with filter on `send-time`;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.sqlpp
new file mode 100644
index 0000000..4d6df66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+load  dataset FacebookMessages using localfs ((`path`=`asterix_nc1://data/fbm-with-send-time.adm`),(`format`=`adm`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.sqlpp
new file mode 100644
index 0000000..b2fcb2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.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.
+ */
+
+use test;
+
+
+create  index fbAuthorIdx  on FacebookMessages (`author-id`) type btree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.sqlpp
new file mode 100644
index 0000000..ed72fe1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select m.`message-id`, m.`author-id`
+from  FacebookMessages as m
+where m.`author-id` >= 1
+order by m.`message-id`;


[11/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-06.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-06.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-06.aql
new file mode 100644
index 0000000..d4e4d46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-06.aql
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"point":$o.point}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-07.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-07.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-07.aql
new file mode 100644
index 0000000..6564565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-07.aql
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"rec":$o, "id":$o.id}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-08.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-08.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-08.aql
new file mode 100644
index 0000000..6c67c49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-08.aql
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return $o
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-09.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-09.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-09.aql
new file mode 100644
index 0000000..d6ec571
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-09.aql
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+let $area:=create-circle(create-point(4.0,3.0), 5.0)
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-index-only.aql
new file mode 100644
index 0000000..54bcef1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-index-only.aql
@@ -0,0 +1,51 @@
+/*
+ * 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 dataverse test;
+
+create type MyRecord as closed {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create nodegroup group1  if not exists on asterix_nc1, asterix_nc2;
+
+create dataset MyData(MyRecord)
+  primary key id on group1;
+
+load dataset MyData 
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+create index rtree_index_point on MyData(point) type rtree;
+
+
+write output to asterix_nc1:"rttest/index_rtree-secondary-index.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open-index-only.aql
new file mode 100644
index 0000000..8e7420e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open-index-only.aql
@@ -0,0 +1,51 @@
+/*
+ * 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 dataverse test;
+
+create type MyRecord as open {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create nodegroup group1  if not exists on asterix_nc1, asterix_nc2;
+
+create dataset MyData(MyRecord)
+  primary key id on group1;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+create index rtree_index_point on MyData(point) type rtree;
+
+
+write output to asterix_nc1:"rttest/index_rtree-secondary-index-open.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open.aql
index 8e7420e..3372e64 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index-open.aql
@@ -43,8 +43,7 @@ using localfs
 
 create index rtree_index_point on MyData(point) type rtree;
 
-
-write output to asterix_nc1:"rttest/index_rtree-secondary-index-open.adm";
+set noindexonly 'true';
 
 for $o in dataset('MyData')
 where spatial-intersect($o.point, create-polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index.aql
index 54bcef1..84aad24 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-secondary-index.aql
@@ -37,14 +37,13 @@ create nodegroup group1  if not exists on asterix_nc1, asterix_nc2;
 create dataset MyData(MyRecord)
   primary key id on group1;
 
-load dataset MyData 
+load dataset MyData
 using localfs
 (("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
 
 create index rtree_index_point on MyData(point) type rtree;
 
-
-write output to asterix_nc1:"rttest/index_rtree-secondary-index.adm";
+set noindexonly 'true';
 
 for $o in dataset('MyData')
 where spatial-intersect($o.point, create-polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019-index-only.aql
new file mode 100644
index 0000000..7fc67ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019-index-only.aql
@@ -0,0 +1,75 @@
+/*
+ * 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 emergencyTest if exists;
+create dataverse emergencyTest;
+
+use dataverse emergencyTest;
+
+create type CHPReport as
+{ "rid":uuid, "etype":string, "timestamp":datetime, "epicenter":point, "radius":double, "message":string }
+
+create type userLocation as
+{ "id":uuid, "user-id":int64, "timestamp":datetime, "location":point }
+
+create type tornadoShelter as
+{ "tsid":uuid, "name":string, "location":point }
+
+create dataset tornadoShelters(tornadoShelter)
+primary key tsid autogenerated;
+
+create dataset userLocations(userLocation)
+primary key id autogenerated;
+
+create dataset CHPReports(CHPReport)
+primary key rid autogenerated;
+
+create index reportTimes on CHPReports(timestamp);
+create index userTimes on userLocations(timestamp);
+create index shelterloc on tornadoShelters(location) type rtree;
+
+create function NearbySheltersDuringTornadoDanger($userid) {
+for $emergency in dataset CHPReports
+let $dangerzone := create-circle($emergency.epicenter,$emergency.radius)
+let $timewindow := day-time-duration("PT1M")
+where (some $user in dataset userLocations satisfies
+$user.user-id = $userid
+)
+return
+{ "shelter locations":for $shelter in dataset tornadoShelters where spatial-intersect( $dangerzone,$shelter.location) return $shelter.location}
+};
+
+create type sub as closed
+{ "id":int, "param0":int64 }
+
+create dataset NearbySheltersDuringTornadoDangerChannelSubscriptions(sub)
+primary key id;
+
+create type result as open
+{ "id": uuid }
+
+create dataset NearbySheltersDuringTornadoDangerChannelResults(result)
+primary key id autogenerated;
+
+insert into dataset NearbySheltersDuringTornadoDangerChannelResults (
+for $sub in dataset NearbySheltersDuringTornadoDangerChannelSubscriptions
+for $result in NearbySheltersDuringTornadoDanger($sub.param0)
+return
+{ "result":$result}
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019.aql
index 7fc67ea..ab6374a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1019.aql
@@ -67,6 +67,8 @@ create type result as open
 create dataset NearbySheltersDuringTornadoDangerChannelResults(result)
 primary key id autogenerated;
 
+set noindexonly 'true';
+
 insert into dataset NearbySheltersDuringTornadoDangerChannelResults (
 for $sub in dataset NearbySheltersDuringTornadoDangerChannelSubscriptions
 for $result in NearbySheltersDuringTornadoDanger($sub.param0)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.plan
new file mode 100644
index 0000000..0118285
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$46(ASC), $$21(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.plan
new file mode 100644
index 0000000..d10b02e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.plan
@@ -0,0 +1,45 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        -- AGGREGATE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
index 9f60440..36bd947 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -16,15 +16,15 @@
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$19][$$15]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$40][$$36]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
index f372412..9f70298 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
index d7a7847..1917413 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -17,13 +17,13 @@
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$30]  |PARTITIONED|
+                                      -- SORT_GROUP_BY[$$64]  |PARTITIONED|
                                               {
                                                 -- AGGREGATE  |LOCAL|
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
-                                        -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$23]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$57]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.plan
new file mode 100644
index 0000000..562bd25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        -- AGGREGATE  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+                          -- UNION_ALL  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SPLIT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
new file mode 100644
index 0000000..98f4731
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
@@ -0,0 +1,81 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        -- AGGREGATE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SPLIT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- SPLIT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- UNION_ALL  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SPLIT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- SPLIT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
new file mode 100644
index 0000000..0ad315e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.plan
@@ -0,0 +1,65 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        -- AGGREGATE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
new file mode 100644
index 0000000..71ee3db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-composite-index-indexonly-plan-01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-composite-index-indexonly-plan-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-composite-index-indexonly-plan-01.plan
new file mode 100644
index 0000000..1b65e5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-composite-index-indexonly-plan-01.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SPLIT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
new file mode 100644
index 0000000..e9bccc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01.plan
new file mode 100644
index 0000000..db61d44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-01.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-02.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-02.plan
new file mode 100644
index 0000000..cabaf8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-02.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$7(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-03.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-03.plan
new file mode 100644
index 0000000..8b23f70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-03.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- UNION_ALL  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-04.plan
new file mode 100644
index 0000000..0151c9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-04.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SPLIT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-05.plan
new file mode 100644
index 0000000..5e8ca40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-05.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$11(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-06.plan
new file mode 100644
index 0000000..3af5b45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-06.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+                -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- UNION_ALL  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SPLIT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-07.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-07.plan
new file mode 100644
index 0000000..5e8ca40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-07.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$11(ASC) ]  |PARTITIONED|
+              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- UNION_ALL  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SPLIT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-08.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-08.plan
new file mode 100644
index 0000000..000efdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-index-indexonly-plan-08.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- AGGREGATE  |UNPARTITIONED|
+      -- AGGREGATE  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$9(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[04/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
new file mode 100644
index 0000000..37358b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
new file mode 100644
index 0000000..33888ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {"pk":o.id, "sk":o.point}
+from MyData o
+where o.docid < 10 and o.val1 < 3) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
new file mode 100644
index 0000000..fe845b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.sqlpp
new file mode 100644
index 0000000..234ec09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select o
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.sqlpp
new file mode 100644
index 0000000..d83edac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-circle`(test.`create-point`(4.0,3.0), 5.0))
+select element {"id":o.id}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.sqlpp
new file mode 100644
index 0000000..aad34ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+
+load dataset Fragile_raw using localfs
+(("path"="asterix_nc1://data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.sqlpp
new file mode 100644
index 0000000..437adc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select x
+from  Fragile_raw as x
+where test.`spatial-intersect`(x.location, test.`create-polygon`([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.sqlpp
new file mode 100644
index 0000000..b01f08f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select element {"id":o.id}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.sqlpp
new file mode 100644
index 0000000..c6bc5ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select element {"point":o.point}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.sqlpp
new file mode 100644
index 0000000..2e38a96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select element {"id":o.id, "point":o.point}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.sqlpp
new file mode 100644
index 0000000..fa5d2e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select element {"id":o.id}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.sqlpp
new file mode 100644
index 0000000..6bee297
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select element {"point":o.point}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.sqlpp
new file mode 100644
index 0000000..da070e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select element {"rec":o, "id":o.id}
+from MyData as o
+where test.`spatial-intersect`(o.point, area)
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
new file mode 100644
index 0000000..f5230ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary full-text index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we have multiple conditions that one index can cover.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
new file mode 100644
index 0000000..bc705b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
new file mode 100644
index 0000000..8c1b943
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with area as (test.`create-rectangle`(point("0.0,0.0"), point("4.0,4.0")))
+select value count(*) from (
+select element {"id":o.id, "point":o.point}
+from MyData as o
+where test.`spatial-intersect`(o.point, area) and o.val1 < 30
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.sqlpp
new file mode 100644
index 0000000..f8a0b48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.ddl.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary full-text index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we have a condition that uses a R-Tree index on polygon fields.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index rtree_index_polygon on MyData(poly1) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.sqlpp
new file mode 100644
index 0000000..bc705b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.sqlpp
new file mode 100644
index 0000000..35c4850
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+with ps as [test.`point`("1.0,1.0"), test.`point`("3.0,3.0")]
+select value count(*) from (
+select element {"id":o.id, "point":o.point}
+from ps as p, MyData as o
+where test.`spatial-intersect`(p, o.poly1)
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.adm
new file mode 100644
index 0000000..39b8897
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.adm
@@ -0,0 +1,15 @@
+{ "message-id": 1, "author-id": 3 }
+{ "message-id": 2, "author-id": 1 }
+{ "message-id": 3, "author-id": 2 }
+{ "message-id": 4, "author-id": 1 }
+{ "message-id": 5, "author-id": 6 }
+{ "message-id": 6, "author-id": 2 }
+{ "message-id": 7, "author-id": 5 }
+{ "message-id": 8, "author-id": 1 }
+{ "message-id": 9, "author-id": 3 }
+{ "message-id": 10, "author-id": 1 }
+{ "message-id": 11, "author-id": 1 }
+{ "message-id": 12, "author-id": 10 }
+{ "message-id": 13, "author-id": 10 }
+{ "message-id": 14, "author-id": 9 }
+{ "message-id": 15, "author-id": 7 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.adm
new file mode 100644
index 0000000..e706c60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.adm
@@ -0,0 +1,5 @@
+{ "aid": 5, "bid": 98, "title": "Active Database Systems" }
+{ "aid": 34, "bid": 57, "title": "Java in Next-Generation Database Systems" }
+{ "aid": 54, "bid": 91, "title": "A Shared View of Sharing  The Treaty of Orlando" }
+{ "aid": 68, "bid": 57, "title": "Java in Next-Generation Database Systems" }
+{ "aid": 69, "bid": 57, "title": "Java in Next-Generation Database Systems" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.adm
deleted file mode 100644
index d0c6981..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.adm
+++ /dev/null
@@ -1,44 +0,0 @@
-{ "aid": 1, "bid": 17, "apt": point("4.1,7.0"), "bp": point("4.1,7.0") }
-{ "aid": 3, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 3, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 3, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 3, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 3, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 4, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 4, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 4, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 4, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 4, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 5, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 5, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 5, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 5, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 5, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 6, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 6, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 6, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 6, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 6, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 7, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 7, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 7, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 7, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 7, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 8, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 8, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 8, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 8, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 8, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
-{ "aid": 15, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 15, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 15, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 16, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 16, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 16, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 17, "bid": 1, "apt": point("4.1,7.0"), "bp": point("4.1,7.0") }
-{ "aid": 18, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 18, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 18, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 19, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 19, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
-{ "aid": 19, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.adm
new file mode 100644
index 0000000..d0c6981
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.adm
@@ -0,0 +1,44 @@
+{ "aid": 1, "bid": 17, "apt": point("4.1,7.0"), "bp": point("4.1,7.0") }
+{ "aid": 3, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 8, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 3, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 4, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 5, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 6, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 7, "apt": point("43.5083,-79.3007"), "bp": point("43.5083,-79.3007") }
+{ "aid": 15, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 15, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 15, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 16, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 16, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 16, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 17, "bid": 1, "apt": point("4.1,7.0"), "bp": point("4.1,7.0") }
+{ "aid": 18, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 18, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 18, "bid": 19, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 19, "bid": 15, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 19, "bid": 16, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }
+{ "aid": 19, "bid": 18, "apt": point("-2.0,3.0"), "bp": point("-2.0,3.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.adm
new file mode 100644
index 0000000..1503808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.adm
@@ -0,0 +1,65 @@
+{ "aid": 1, "bid": 1, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 1, "bid": 17, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 2, "bid": 2, "apoint": point("40.2152,-75.0449"), "bpoint": point("40.2152,-75.0449") }
+{ "aid": 3, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 9, "bid": 9, "apoint": point("5.0,1.0"), "bpoint": point("5.0,1.0") }
+{ "aid": 10, "bid": 10, "apoint": point("2.0,3.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 11, "bid": 11, "apoint": point("4.9,0.0"), "bpoint": point("4.9,0.0") }
+{ "aid": 12, "bid": 12, "apoint": point("6.0,3.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 13, "bid": 13, "apoint": point("5.0,5.0"), "bpoint": point("5.0,5.0") }
+{ "aid": 14, "bid": 14, "apoint": point("5.1,5.1"), "bpoint": point("5.1,5.1") }
+{ "aid": 15, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 17, "bid": 1, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 17, "bid": 17, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 18, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 20, "bid": 20, "apoint": point("4.0,3.0"), "bpoint": point("4.0,3.0") }
+{ "aid": 21, "bid": 21, "apoint": point("0.0,5.0"), "bpoint": point("0.0,5.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.adm
new file mode 100644
index 0000000..1503808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.adm
@@ -0,0 +1,65 @@
+{ "aid": 1, "bid": 1, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 1, "bid": 17, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 2, "bid": 2, "apoint": point("40.2152,-75.0449"), "bpoint": point("40.2152,-75.0449") }
+{ "aid": 3, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 3, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 4, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 5, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 6, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 7, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 3, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 4, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 5, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 6, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 7, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 8, "bid": 8, "apoint": point("43.5083,-79.3007"), "bpoint": point("43.5083,-79.3007") }
+{ "aid": 9, "bid": 9, "apoint": point("5.0,1.0"), "bpoint": point("5.0,1.0") }
+{ "aid": 10, "bid": 10, "apoint": point("2.0,3.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 11, "bid": 11, "apoint": point("4.9,0.0"), "bpoint": point("4.9,0.0") }
+{ "aid": 12, "bid": 12, "apoint": point("6.0,3.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 13, "bid": 13, "apoint": point("5.0,5.0"), "bpoint": point("5.0,5.0") }
+{ "aid": 14, "bid": 14, "apoint": point("5.1,5.1"), "bpoint": point("5.1,5.1") }
+{ "aid": 15, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 15, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 16, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 17, "bid": 1, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 17, "bid": 17, "apoint": point("4.1,7.0"), "bpoint": point("4.1,7.0") }
+{ "aid": 18, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 18, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 15, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 16, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 18, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 19, "bid": 19, "apoint": point("-2.0,3.0"), "bpoint": point("-2.0,3.0") }
+{ "aid": 20, "bid": 20, "apoint": point("4.0,3.0"), "bpoint": point("4.0,3.0") }
+{ "aid": 21, "bid": 21, "apoint": point("0.0,5.0"), "bpoint": point("0.0,5.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.adm
new file mode 100644
index 0000000..5e2281b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.adm
@@ -0,0 +1,20 @@
+{ "aid": 1, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 2, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 3, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 4, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 6, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 9, "bid": 10, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 11, "bid": 9, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("5.0,1.0") }
+{ "aid": 11, "bid": 12, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 12, "bid": 12, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 12, "bid": 20, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("4.0,3.0") }
+{ "aid": 13, "bid": 12, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 14, "bid": 12, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 15, "bid": 12, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 16, "bid": 12, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 17, "bid": 10, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoint": point("2.0,3.0") }
+{ "aid": 17, "bid": 12, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 17, "bid": 20, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoint": point("4.0,3.0") }
+{ "aid": 17, "bid": 21, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoint": point("0.0,5.0") }
+{ "aid": 20, "bid": 12, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("6.0,3.0") }
+{ "aid": 20, "bid": 20, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoint": point("4.0,3.0") }


[15/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 10630a5..9950e37 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -20,8 +20,12 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
@@ -50,11 +54,14 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Quadruple;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -68,15 +75,21 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCall
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
@@ -89,6 +102,13 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8S
  */
 public class AccessMethodUtils {
 
+    // Output variable type from a secondary unnest-map
+    enum SecondaryUnnestMapOutputVarType {
+        PRIMARY_KEY,
+        SECONDARY_KEY,
+        CONDITIONAL_SPLIT_VAR
+    }
+
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
             List<Object> target) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
@@ -292,7 +312,9 @@ public class AccessMethodUtils {
      * Appends the types of the fields produced by the given secondary index to dest.
      */
     public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, ARecordType metaRecordType,
-            Index index, boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+            Index index, List<Object> dest, boolean requireResultOfInstantTryLock) throws AlgebricksException {
+        // In case of an inverted-index search, secondary keys will not be generated.
+        boolean primaryKeysOnly = isInvertedIndex(index);
         if (!primaryKeysOnly) {
             switch (index.getIndexType()) {
                 case BTREE:
@@ -301,10 +323,6 @@ public class AccessMethodUtils {
                 case RTREE:
                     dest.addAll(KeyFieldTypeUtil.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
-                case SINGLE_PARTITION_WORD_INVIX:
-                case SINGLE_PARTITION_NGRAM_INVIX:
-                case LENGTH_PARTITIONED_NGRAM_INVIX:
-                case LENGTH_PARTITIONED_WORD_INVIX:
                 default:
                     break;
             }
@@ -316,12 +334,25 @@ public class AccessMethodUtils {
         } else {
             dest.addAll(KeyFieldTypeUtil.getPartitoningKeyTypes(dataset, recordType, metaRecordType));
         }
+
+        // Adds one more type to apply an index-only plan optimization.
+        // Currently, we use AINT32 to decode result values for this.
+        // Refer to appendSecondaryIndexOutputVars() for more details.
+        if (requireResultOfInstantTryLock) {
+            dest.add(BuiltinType.AINT32);
+        }
+
     }
 
+    /**
+     * Creates output variables for the given unnest-map or left-outer-unnestmap operator
+     * that does a secondary index lookup.
+     * The order: SK, PK, [Optional: the result of a instantTryLock on PK]
+     */
     public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType,
-            ARecordType metaRecordType, Index index, boolean primaryKeysOnly, IOptimizationContext context,
-            List<LogicalVariable> dest) throws AlgebricksException {
-        int numPrimaryKeys = 0;
+            ARecordType metaRecordType, Index index, IOptimizationContext context, List<LogicalVariable> dest,
+            boolean requireResultOfInstantTryLock) throws AlgebricksException {
+        int numPrimaryKeys;
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             numPrimaryKeys = IndexingConstants
                     .getRIDSize(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
@@ -329,33 +360,80 @@ public class AccessMethodUtils {
             numPrimaryKeys = dataset.getPrimaryKeys().size();
         }
         int numSecondaryKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, recordType, metaRecordType);
-        int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
+        // In case of an inverted-index search, secondary keys will not be generated.
+        int numVars = isInvertedIndex(index) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
+
+        // If it's an index-only plan, add one more variable to put the result of instantTryLock on PK -
+        // whether this lock can be granted on a primary key.
+        // If it is granted, then we don't need to do a post verification (select).
+        // If it is not granted, then we need to do a secondary index lookup, do a primary index lookup, and select.
+        if (requireResultOfInstantTryLock) {
+            numVars += 1;
+        }
+
         for (int i = 0; i < numVars; i++) {
             dest.add(context.newVar());
         }
     }
 
-    public static List<LogicalVariable> getPrimaryKeyVarsFromSecondaryUnnestMap(Dataset dataset,
-            ILogicalOperator unnestMapOp) {
+    /**
+     * Gets the primary key variables from the unnest-map or left-outer-unnest-map operator
+     * that does a secondary index lookup.
+     * The order: SK, PK, [Optional: the result of a TryLock on PK]
+     */
+    public static List<LogicalVariable> getKeyVarsFromSecondaryUnnestMap(Dataset dataset, ARecordType recordType,
+            ARecordType metaRecordType, ILogicalOperator unnestMapOp, Index index,
+            SecondaryUnnestMapOutputVarType keyVarType) throws AlgebricksException {
         int numPrimaryKeys;
+        int numSecondaryKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, recordType, metaRecordType);
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             numPrimaryKeys = IndexingConstants
                     .getRIDSize(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
         } else {
             numPrimaryKeys = dataset.getPrimaryKeys().size();
         }
-        List<LogicalVariable> primaryKeyVars = new ArrayList<>();
-        List<LogicalVariable> sourceVars = null;
-
-        sourceVars = ((AbstractUnnestMapOperator) unnestMapOp).getVariables();
+        List<LogicalVariable> keyVars = new ArrayList<>();
+        AbstractUnnestMapOperator abstractUnnestMapOp = (AbstractUnnestMapOperator) unnestMapOp;
+        List<LogicalVariable> sourceVars = abstractUnnestMapOp.getVariables();
+        // Assumption: the primary keys are located after the secondary key.
+        int start;
+        int stop;
+
+        // If a secondary-index search didn't generate SKs, set it to zero.
+        // Currently, only an inverted-index search doesn't generate any SKs.
+        boolean isNgramOrKeywordIndex = isInvertedIndex(index);
+        if (isNgramOrKeywordIndex) {
+            numSecondaryKeys = 0;
+        }
 
-        // Assumes the primary keys are located at the end.
-        int start = sourceVars.size() - numPrimaryKeys;
-        int stop = sourceVars.size();
+        // Fetches keys: type 0 - PK, type 1 - SK, type 2 - the result of instantTryLock() on PK
+        switch (keyVarType) {
+            case PRIMARY_KEY:
+                // Fetches primary keys - the second position
+                start = numSecondaryKeys;
+                stop = numSecondaryKeys + numPrimaryKeys;
+                break;
+            case SECONDARY_KEY:
+                // Fetches secondary keys - the first position
+                start = 0;
+                stop = numSecondaryKeys;
+                break;
+            case CONDITIONAL_SPLIT_VAR:
+                // Sanity check - the given unnest map should generate this variable.
+                if (!abstractUnnestMapOp.getGenerateCallBackProceedResultVar()) {
+                    throw CompilationException.create(ErrorCode.CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE);
+                }
+                // Fetches conditional splitter - the last position
+                start = numSecondaryKeys + numPrimaryKeys;
+                stop = start + 1;
+                break;
+            default:
+                return Collections.emptyList();
+        }
         for (int i = start; i < stop; i++) {
-            primaryKeyVars.add(sourceVars.get(i));
+            keyVars.add(sourceVars.get(i));
         }
-        return primaryKeyVars;
+        return keyVars;
     }
 
     public static List<LogicalVariable> getPrimaryKeyVarsFromPrimaryUnnestMap(Dataset dataset,
@@ -384,10 +462,9 @@ public class AccessMethodUtils {
      *
      * @throws AlgebricksException
      */
-    public static Pair<ILogicalExpression, Boolean> createSearchKeyExpr(Index index, IOptimizableFuncExpr optFuncExpr,
-            IAType indexedFieldType, OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree)
+    public static Triple<ILogicalExpression, ILogicalExpression, Boolean> createSearchKeyExpr(Index index,
+            IOptimizableFuncExpr optFuncExpr, IAType indexedFieldType, OptimizableOperatorSubTree probeSubTree)
             throws AlgebricksException {
-
         if (probeSubTree == null) {
             // We are optimizing a selection query. Search key is a constant.
             // Type Checking and type promotion is done here
@@ -396,7 +473,7 @@ public class AccessMethodUtils {
                 //We are looking at a selection case, but using two variables
                 //This means that the second variable comes from a nonPure function call
                 //TODO: Right now we miss on type promotion for nonpure functions
-                return new Pair<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), false);
+                return new Triple<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), null, false);
             }
 
             ILogicalExpression constantAtRuntimeExpression = optFuncExpr.getConstantExpr(0);
@@ -408,25 +485,103 @@ public class AccessMethodUtils {
             ATypeTag constantValueTag = optFuncExpr.getConstantType(0).getTypeTag();
             ATypeTag indexedFieldTypeTag = TypeComputeUtils.getActualType(indexedFieldType).getTypeTag();
 
-            // if the constant type and target type does not match, we do a type conversion
-            AsterixConstantValue replacedConstantValue = null;
             // type casting happened from real (FLOAT, DOUBLE) value -> INT value?
             boolean realTypeConvertedToIntegerType = false;
+            // constant value after type-casting is applied.
+            AsterixConstantValue replacedConstantValue = null;
+            // constant value after type-casting is applied - this value is used only for equality case.
+            // Refer to the following switch case for more details.
+            AsterixConstantValue replacedConstantValueForEQCase = null;
 
+            // If the constant type and target type does not match, we may need to do a type conversion.
             if (constantValueTag != indexedFieldTypeTag && constantValue != null) {
-                try {
-                    replacedConstantValue = ATypeHierarchy.getAsterixConstantValueFromNumericTypeObject(
-                            constantValue.getObject(), indexedFieldTypeTag, index.isEnforced());
-                    realTypeConvertedToIntegerType =
-                            isRealTypeConvertedToIntegerType(constantValueTag, indexedFieldTypeTag);
-                } catch (HyracksDataException e) {
-                    throw new AlgebricksException(e);
+                // To check whether the constant is REAL values, and target field is an INT type field.
+                // In this case, we need to change the search parameter. Refer to the caller section for the detail.
+                realTypeConvertedToIntegerType =
+                        isRealTypeConvertedToIntegerType(constantValueTag, indexedFieldTypeTag);
+                if (realTypeConvertedToIntegerType && !index.isEnforced() && !index.isOverridingKeyFieldTypes()) {
+                    // For the index on a closed-type field,
+                    // if a DOUBLE or FLOAT constant is converted to an INT type value,
+                    // we need to check a corner case where two real values are located
+                    // between an INT value. For example, the following query,
+                    //
+                    // for $emp in dataset empDataset
+                    // where $emp.age > double("2.3") and $emp.age < double("3.3")
+                    // return $emp.id;
+                    //
+                    // It should generate a result if there is a tuple that satisfies the condition,
+                    // which is 3. However, it does not generate the desired result since finding
+                    // candidates fails after truncating the fraction part
+                    // (there is no INT whose value is greater than 2 and less than 3.)
+                    //
+                    // Thus,
+                    // when converting FLOAT or DOUBLE values, we need to apply ceil() or floor().
+                    // The following transformation will generate the final result, not a superset of it.
+                    //
+                    // LT
+                    // IntVar < 4.9 ==> round-up: IntVar < 5
+                    //
+                    // LE
+                    // IntVar <= 4.9  ==> round-down: IntVar <= 4
+                    //
+                    // GT
+                    // IntVar > 4.9 ==> round-down: IntVar > 4
+                    //
+                    // GE
+                    // IntVar >= 4.9 ==> round-up: IntVar >= 5
+                    //
+                    // EQ: two values are required to do a correct type-casting.
+                    // IntVar = 4.3 ==> round-down and round-up: IntVar = 4 and IntVar = 5 : false
+                    // IntVar = 4.0 ==> round-down and round-up: IntVar = 4 and IntVar = 4 : true
+                    FunctionIdentifier functionID = optFuncExpr.getFuncExpr().getFunctionIdentifier();
+                    ComparisonKind cKind = AlgebricksBuiltinFunctions.getComparisonType(functionID);
+
+                    switch (cKind) {
+                        case LT:
+                        case GE:
+                            // round-up
+                            replacedConstantValue =
+                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            break;
+                        case LE:
+                        case GT:
+                            // round-down
+                            replacedConstantValue =
+                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
+                            break;
+                        case EQ:
+                            // equality case - both CEIL and FLOOR need to be applied.
+                            replacedConstantValue =
+                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
+                            replacedConstantValueForEQCase =
+                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            break;
+                        default:
+                            // NEQ should not be a case.
+                            throw new IllegalStateException();
+                    }
+                } else if (!realTypeConvertedToIntegerType) {
+                    // Type conversion only case: (e.g., INT -> BIGINT)
+                    replacedConstantValue = getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.NONE);
                 }
             }
-
-            return replacedConstantValue != null
-                    ? new Pair<>(new ConstantExpression(replacedConstantValue), realTypeConvertedToIntegerType)
-                    : new Pair<>(constantAtRuntimeExpression, false);
+            // No type-casting at all
+            if (replacedConstantValue == null) {
+                return new Triple<>(constantAtRuntimeExpression, null, false);
+            }
+            // A type-casting happened, but not EQ case
+            if (replacedConstantValueForEQCase == null) {
+                return new Triple<>(new ConstantExpression(replacedConstantValue), null,
+                        realTypeConvertedToIntegerType);
+            }
+            // A type-casting happened and it's an EQ case.
+            return new Triple<>(new ConstantExpression(replacedConstantValue),
+                    new ConstantExpression(replacedConstantValueForEQCase), realTypeConvertedToIntegerType);
         } else {
             // We are optimizing a join query. Determine which variable feeds the secondary index.
             OptimizableOperatorSubTree opSubTree0 = optFuncExpr.getOperatorSubTree(0);
@@ -445,11 +600,22 @@ public class AccessMethodUtils {
                     TypeCastUtils.setRequiredAndInputTypes(castFunc, indexedFieldType, probeType);
                     boolean realTypeConvertedToIntegerType =
                             isRealTypeConvertedToIntegerType(probeTypeTypeTag, indexedFieldTypeTag);
-                    return new Pair<>(castFunc, realTypeConvertedToIntegerType);
+                    return new Triple<>(castFunc, null, realTypeConvertedToIntegerType);
                 }
             }
+            return new Triple<>(probeExpr, null, false);
+        }
+    }
 
-            return new Pair<>(probeExpr, false);
+    private static AsterixConstantValue getReplacedConstantValue(IAObject sourceObject, ATypeTag sourceTypeTag,
+            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction)
+            throws CompilationException {
+        try {
+            return ATypeHierarchy.getAsterixConstantValueFromNumericTypeObject(sourceObject, targetTypeTag,
+                    strictDemote, mathFunction);
+        } catch (HyracksDataException e) {
+            throw new CompilationException(ErrorCode.ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION, e, sourceTypeTag,
+                    targetTypeTag);
         }
     }
 
@@ -490,10 +656,119 @@ public class AccessMethodUtils {
         return indexExprs.get(0).second;
     }
 
+    /**
+     * Checks whether the given function expression can utilize the given index.
+     * If so, checks the given join plan is an index-only plan or not.
+     */
+    public static boolean setIndexOnlyPlanInfo(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree probeSubTree,
+            OptimizableOperatorSubTree indexSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, AbstractFunctionCallExpression funcExpr,
+            List<Pair<FunctionIdentifier, Boolean>> funcIdentifiers) throws AlgebricksException {
+        // index-only plan possible?
+        boolean isIndexOnlyPlan = false;
+
+        // Whether a verification (especially for R-Tree case) is required after the secondary index search
+        // In other words, can the chosen method generate any false positive results?
+        // Currently, for the B+ Tree index, there cannot be any false positive results except the composite index case.
+        boolean requireVerificationAfterSIdxSearch = false;
+
+        // Does the given index can cover all search predicates?
+        boolean doesSIdxSearchCoverAllPredicates = false;
+
+        Pair<Boolean, Boolean> functionFalsePositiveCheck =
+                AccessMethodUtils.canFunctionGenerateFalsePositiveResultsUsingIndex(funcExpr, funcIdentifiers);
+
+        if (functionFalsePositiveCheck.first) {
+            requireVerificationAfterSIdxSearch = functionFalsePositiveCheck.second;
+        } else {
+            // Function not found?
+            return false;
+        }
+
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = new Quadruple<>(isIndexOnlyPlan, false,
+                requireVerificationAfterSIdxSearch, doesSIdxSearchCoverAllPredicates);
+
+        if (analysisCtx.getIndexDatasetMap().get(chosenIndex).getDatasetType() == DatasetType.INTERNAL) {
+            AccessMethodUtils.indexOnlyPlanCheck(afterJoinRefs, joinRef, indexSubTree, probeSubTree, chosenIndex,
+                    analysisCtx, context, indexOnlyPlanInfo);
+        } else {
+            // We don't consider an index on an external dataset to be an index-only plan.
+            isIndexOnlyPlan = false;
+            indexOnlyPlanInfo.setFirst(isIndexOnlyPlan);
+        }
+
+        analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
+
+        return true;
+    }
+
+    /**
+     * Finalizes the index-nested-loop join plan transformation.
+     */
+    public static boolean finalizeJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
+            boolean hasGroupBy, ILogicalOperator indexSearchOp, LogicalVariable newNullPlaceHolderVar,
+            Mutable<ILogicalExpression> conditionRef, Dataset dataset) throws AlgebricksException {
+        ILogicalOperator finalIndexSearchOp = indexSearchOp;
+        if (isLeftOuterJoin && hasGroupBy) {
+            ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingFuncInGroupBy();
+            List<LogicalVariable> lojMissingVariables = new ArrayList<>();
+            lojFuncExprs.getUsedVariables(lojMissingVariables);
+            boolean lojMissingVarExist = false;
+            if (!lojMissingVariables.isEmpty()) {
+                lojMissingVarExist = true;
+            }
+
+            // Resets the missing place holder variable.
+            AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+
+            // For the index-only plan, if newNullPlaceHolderVar is not in the variable map of the union operator
+            // or if the variable is removed during the above method, we need to refresh the variable mapping in UNION.
+            finalIndexSearchOp = AccessMethodUtils.resetVariableMappingInUnionOpInIndexOnlyPlan(lojMissingVarExist,
+                    lojMissingVariables, indexSearchOp, afterJoinRefs, context);
+        }
+
+        boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
+        // If there are any left conditions, add a new select operator on top.
+        indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
+        if (conditionRef.getValue() != null) {
+            // If an index-only plan is possible, the whole plan is now changed.
+            // Replaces the current path with the new index-only plan.
+            if (isIndexOnlyPlan && dataset.getDatasetType() == DatasetType.INTERNAL) {
+                // Gets the revised dataSourceRef operator from the secondary index-search.
+                ILogicalOperator dataSourceRefOp =
+                        AccessMethodUtils.findDataSourceFromIndexUtilizationPlan(finalIndexSearchOp);
+                if (dataSourceRefOp != null && (dataSourceRefOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP
+                        || dataSourceRefOp.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP)) {
+                    indexSubTree.getDataSourceRef().setValue(dataSourceRefOp);
+                }
+                // Replaces the current operator with the newly created UNIONALL operator.
+                joinRef.setValue(finalIndexSearchOp);
+            } else {
+                // Non-index only plan case
+                indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
+                SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+                topSelectOp.getInputs().add(indexSubTree.getRootRef());
+                topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
+                context.computeAndSetTypeEnvironmentForOperator(topSelectOp);
+                joinRef.setValue(topSelectOp);
+            }
+        } else {
+            if (finalIndexSearchOp.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
+                joinRef.setValue(finalIndexSearchOp);
+            } else {
+                joinRef.setValue(indexSubTree.getRootRef().getValue());
+            }
+        }
+        return true;
+    }
+
     public static ILogicalOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType,
             ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
-            IOptimizationContext context, boolean outputPrimaryKeysOnly, boolean retainInput, boolean retainNull)
-            throws AlgebricksException {
+            IOptimizationContext context, boolean retainInput, boolean retainNull,
+            boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
         jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
@@ -501,17 +776,19 @@ public class AccessMethodUtils {
         List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<>();
         List<Object> secondaryIndexOutputTypes = new ArrayList<>();
         // Append output variables/types generated by the secondary-index search (not forwarded from input).
-        appendSecondaryIndexOutputVars(dataset, recordType, metaRecordType, index, outputPrimaryKeysOnly, context,
-                secondaryIndexUnnestVars);
-        appendSecondaryIndexTypes(dataset, recordType, metaRecordType, index, outputPrimaryKeysOnly,
-                secondaryIndexOutputTypes);
+        // Output: SK, PK, [Optional: the result of instantTryLock]
+        appendSecondaryIndexOutputVars(dataset, recordType, metaRecordType, index, context, secondaryIndexUnnestVars,
+                generateInstantTrylockResultFromIndexSearch);
+        appendSecondaryIndexTypes(dataset, recordType, metaRecordType, index, secondaryIndexOutputTypes,
+                generateInstantTrylockResultFromIndexSearch);
         // An index search is expressed as an unnest over an index-search function.
         IFunctionInfo secondaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         UnnestingFunctionCallExpression secondaryIndexSearchFunc =
                 new UnnestingFunctionCallExpression(secondaryIndexSearch, secondaryIndexFuncArgs);
         secondaryIndexSearchFunc.setReturnsUniqueValues(true);
-        // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
-        // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
+        // This is the operator that jobgen will be looking for. It contains an unnest function that has all
+        // necessary arguments to determine which index to use, which variables contain the index-search keys,
+        // what is the original dataset, etc.
 
         // Left-outer-join (retainInput and retainNull) case?
         // Then, we use the LEFT-OUTER-UNNEST-MAP operator instead of unnest-map operator.
@@ -520,6 +797,8 @@ public class AccessMethodUtils {
                 LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
                         secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
                         secondaryIndexOutputTypes, true);
+                secondaryIndexLeftOuterUnnestOp
+                        .setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
                 secondaryIndexLeftOuterUnnestOp.getInputs().add(new MutableObject<>(inputOp));
                 context.computeAndSetTypeEnvironmentForOperator(secondaryIndexLeftOuterUnnestOp);
                 secondaryIndexLeftOuterUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -533,6 +812,7 @@ public class AccessMethodUtils {
             UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes,
                     retainInput);
+            secondaryIndexUnnestOp.setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
             secondaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
             context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
             secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -540,12 +820,11 @@ public class AccessMethodUtils {
         }
     }
 
-    public static AbstractUnnestMapOperator createPrimaryIndexUnnestMap(AbstractDataSourceOperator dataSourceOp,
-            Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
-            IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainNull,
-            boolean requiresBroadcast) throws AlgebricksException {
-        List<LogicalVariable> primaryKeyVars =
-                AccessMethodUtils.getPrimaryKeyVarsFromSecondaryUnnestMap(dataset, inputOp);
+    private static AbstractUnnestMapOperator createFinalNonIndexOnlySearchPlan(Dataset dataset,
+            ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
+            boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
+            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
+            throws AlgebricksException {
         // Optionally add a sort on the primary-index keys before searching the primary index.
         OrderOperator order = null;
         if (sortPrimaryKeys) {
@@ -559,6 +838,493 @@ public class AccessMethodUtils {
             order.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(order);
         }
+        // Creates the primary-index search unnest-map operator.
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
+                retainMissing, requiresBroadcast, primaryKeyVars, primaryIndexUnnestVars, primaryIndexOutputTypes);
+        if (sortPrimaryKeys) {
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+        } else {
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
+        }
+        context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
+        primaryIndexUnnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
+        return primaryIndexUnnestMapOp;
+    }
+
+    private static ILogicalOperator createFinalIndexOnlySearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignsBeforeTopOpRef, Dataset dataset, ARecordType recordType,
+            ARecordType metaRecordType, ILogicalOperator inputOp, IOptimizationContext context, boolean retainInput,
+            boolean retainMissing, boolean requiresBroadcast, Index secondaryIndex,
+            AccessMethodAnalysisContext analysisCtx, OptimizableOperatorSubTree subTree,
+            LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
+            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
+            throws AlgebricksException {
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
+        // From now on, we deal with the index-only plan.
+        // Initializes the information required for the index-only plan optimization.
+        // Fetches SK variable(s) from the secondary-index search operator.
+        List<LogicalVariable> skVarsFromSIdxUnnestMap = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset,
+                recordType, metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.SECONDARY_KEY);
+        boolean skFieldUsedAfterTopOp = indexOnlyPlanInfo.getSecond();
+        boolean requireVerificationAfterSIdxSearch = indexOnlyPlanInfo.getThird();
+        ILogicalOperator assignBeforeTopOp;
+        UnionAllOperator unionAllOp;
+        SelectOperator newSelectOpInLeftPath;
+        SelectOperator newSelectOpInRightPath;
+        SplitOperator splitOp = null;
+        // This variable map will be used as input to UNIONALL operator. The form is <left, right, output>.
+        // In our case, left: instantTryLock fail path, right: instantTryLock success path
+        List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> unionVarMap = new ArrayList<>();
+        List<LogicalVariable> condSplitVars;
+        List<LogicalVariable> liveVarsAfterTopOp = new ArrayList<>();
+
+        // Constructs the variable mapping between newly constructed secondary
+        // key search (SK, PK) and those in the original plan (datasource scan).
+        LinkedHashMap<LogicalVariable, LogicalVariable> origVarToSIdxUnnestMapOpVarMap = new LinkedHashMap<>();
+
+        List<List<String>> chosenIndexFieldNames = secondaryIndex.getKeyFieldNames();
+        IndexType idxType = secondaryIndex.getIndexType();
+
+        // variables used in SELECT or JOIN operator
+        List<LogicalVariable> usedVarsInTopOp = new ArrayList<>();
+        List<LogicalVariable> uniqueUsedVarsInTopOp = new ArrayList<>();
+
+        // variables used in ASSIGN before SELECT operator
+        List<LogicalVariable> producedVarsInAssignsBeforeTopOp = new ArrayList<>();
+
+        // For the index-nested-loop join case, we need to exclude the variables from the left (outer) branch
+        // when deciding which variables should be propagated via UNIONALL operator.
+        // This is because these variables are already generated and is not related to the decision
+        // whether the plan is an index-only plan or not. Only the right (inner) branch matters.
+        List<LogicalVariable> liveVarsInSubTreeRootOp = new ArrayList<>();
+
+        // variables used after SELECT or JOIN operator
+        List<LogicalVariable> usedVarsAfterTopOp = new ArrayList<>();
+        List<LogicalVariable> varsTmpList = new ArrayList<>();
+
+        // If the secondary key field is used after SELECT or JOIN operator (e.g., returning the field value),
+        // then we need to keep these secondary keys. In case of R-tree index, the result of an R-tree
+        // index search is an MBR. So, we need to reconstruct original field values from the result if that index
+        // is on a rectangle or point.
+        AssignOperator skVarAssignOpInRightPath = null;
+        List<LogicalVariable> restoredSKVarFromRTree = null;
+        // Original SK field variable to restored SK field variable in the right path mapping
+        LinkedHashMap<LogicalVariable, LogicalVariable> origSKFieldVarToNewSKFieldVarMap = new LinkedHashMap<>();
+        // Index-only plan consideration for the R-Tree index only:
+        // Constructs an additional ASSIGN to restore the original secondary key field(s) from
+        // the results of the secondary index search in case the field is used after SELECT or JOIN operator or
+        // a verification is required since the given query shape is not RECTANGLE or POINT even though the type of
+        // index is RECTANGLE or POINT (in this case only, removing false-positive is possible.).
+        if (idxType == IndexType.RTREE && (skFieldUsedAfterTopOp || requireVerificationAfterSIdxSearch)) {
+            IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(secondaryIndex, analysisCtx);
+            int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(secondaryIndex, analysisCtx);
+            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
+                    optFuncExpr.getFieldName(optFieldIdx), recordType);
+            if (keyPairType == null) {
+                return null;
+            }
+            // Gets the number of dimensions corresponding to the field indexed by chosenIndex.
+            IAType spatialType = keyPairType.first;
+            ArrayList<Mutable<ILogicalExpression>> restoredSKFromRTreeExprs = new ArrayList<>();
+            restoredSKVarFromRTree = new ArrayList<>();
+            switch (spatialType.getTypeTag()) {
+                case POINT:
+                    // Reconstructs a POINT value.
+                    AbstractFunctionCallExpression createPointExpr = createPointExpression(skVarsFromSIdxUnnestMap);
+                    restoredSKVarFromRTree.add(context.newVar());
+                    restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createPointExpr));
+                    skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    break;
+                case RECTANGLE:
+                    // Reconstructs a RECTANGLE value.
+                    AbstractFunctionCallExpression expr1 = createPointExpression(skVarsFromSIdxUnnestMap.subList(0, 2));
+                    AbstractFunctionCallExpression expr2 = createPointExpression(skVarsFromSIdxUnnestMap.subList(2, 4));
+                    AbstractFunctionCallExpression createRectangleExpr = createRectangleExpression(expr1, expr2);
+                    restoredSKVarFromRTree.add(context.newVar());
+                    restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createRectangleExpr));
+                    skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    break;
+                default:
+                    break;
+            }
+        }
+
+        // Gets all variables from the right (inner) branch.
+        VariableUtilities.getLiveVariables((ILogicalOperator) subTree.getRootRef().getValue(), liveVarsInSubTreeRootOp);
+        // Gets the used variables from the SELECT or JOIN operator.
+        VariableUtilities.getUsedVariables((ILogicalOperator) topOpRef.getValue(), usedVarsInTopOp);
+        // Excludes the variables in the condition from the outer branch - in join case.
+        for (Iterator<LogicalVariable> iterator = usedVarsInTopOp.iterator(); iterator.hasNext();) {
+            LogicalVariable v = iterator.next();
+            if (!liveVarsInSubTreeRootOp.contains(v)) {
+                iterator.remove();
+            }
+        }
+        // Keeps the unique used variables in the SELECT or JOIN operator.
+        copyVarsToAnotherList(usedVarsInTopOp, uniqueUsedVarsInTopOp);
+
+        // If there are ASSIGN operators (usually secondary key field) before the given SELECT or JOIN operator,
+        // we may need to propagate these produced variables via the UNIONALL operator if they are used afterwards.
+        if (assignsBeforeTopOpRef != null && !assignsBeforeTopOpRef.isEmpty()) {
+            for (int i = 0; i < assignsBeforeTopOpRef.size(); i++) {
+                assignBeforeTopOp = assignsBeforeTopOpRef.get(i).getValue();
+                varsTmpList.clear();
+                VariableUtilities.getProducedVariables(assignBeforeTopOp, varsTmpList);
+                copyVarsToAnotherList(varsTmpList, producedVarsInAssignsBeforeTopOp);
+            }
+        }
+
+        // Adds an optional ASSIGN operator that sits right after the SELECT or JOIN operator.
+        // This assign operator keeps any constant expression(s) extracted from the original ASSIGN operators
+        // in the subtree and are used after the SELECT or JOIN operator. In usual case,
+        // this constant value would be used in a group-by after a left-outer-join and will be removed by the optimizer.
+        // We need to conduct this since this variable does not have to be in the both branch of an index-only plan.
+        AssignOperator constAssignOp = null;
+        ILogicalOperator currentOpAfterTopOp = null;
+        List<LogicalVariable> constAssignVars = new ArrayList<>();
+        List<Mutable<ILogicalExpression>> constAssignExprs = new ArrayList<>();
+        ILogicalOperator currentOp = inputOp;
+
+        boolean constantAssignVarUsedInTopOp = false;
+        if (assignsBeforeTopOpRef != null) {
+            // From the first ASSIGN (earliest in the plan) to the last ASSGIN (latest)
+            for (int i = assignsBeforeTopOpRef.size() - 1; i >= 0; i--) {
+                AssignOperator tmpOp = (AssignOperator) assignsBeforeTopOpRef.get(i).getValue();
+                List<LogicalVariable> tmpAssignVars = tmpOp.getVariables();
+                List<Mutable<ILogicalExpression>> tmpAsssignExprs = tmpOp.getExpressions();
+                Iterator<LogicalVariable> varIt = tmpAssignVars.iterator();
+                Iterator<Mutable<ILogicalExpression>> exprIt = tmpAsssignExprs.iterator();
+                boolean changed = false;
+                while (exprIt.hasNext()) {
+                    Mutable<ILogicalExpression> tmpExpr = exprIt.next();
+                    LogicalVariable tmpVar = varIt.next();
+                    if (tmpExpr.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                        constAssignVars.add(tmpVar);
+                        constAssignExprs.add(tmpExpr);
+                        varIt.remove();
+                        exprIt.remove();
+                        changed = true;
+                    }
+                }
+                if (changed) {
+                    context.computeAndSetTypeEnvironmentForOperator(tmpOp);
+                }
+            }
+
+            if (!constAssignVars.isEmpty()) {
+                // These constants should not be used in the SELECT or JOIN operator.
+                for (LogicalVariable v : constAssignVars) {
+                    if (usedVarsInTopOp.contains(v)) {
+                        constantAssignVarUsedInTopOp = true;
+                        break;
+                    }
+                }
+                // If this assign operator is not used in the SELECT or JOIN operator,
+                // we will add this operator after creating UNION operator in the last part of this method.
+                constAssignOp = new AssignOperator(constAssignVars, constAssignExprs);
+                if (constantAssignVarUsedInTopOp) {
+                    // Places this assign after the secondary index-search op.
+                    constAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+                    constAssignOp.setExecutionMode(ExecutionMode.PARTITIONED);
+                    context.computeAndSetTypeEnvironmentForOperator(constAssignOp);
+                    currentOp = constAssignOp;
+                }
+            }
+        }
+
+        // variables used after SELECT or JOIN operator
+        HashSet<LogicalVariable> varsTmpSet = new HashSet<>();
+        if (afterTopOpRefs != null) {
+            for (Mutable<ILogicalOperator> afterTopOpRef : afterTopOpRefs) {
+                varsTmpSet.clear();
+                OperatorPropertiesUtil.getFreeVariablesInOp((ILogicalOperator) afterTopOpRef.getValue(), varsTmpSet);
+                copyVarsToAnotherList(varsTmpSet, usedVarsAfterTopOp);
+            }
+        }
+
+        // Now, adds a SPLIT operator to propagate <SK, PK> pair from the secondary-index search to the two paths.
+        // And constructs the path from the secondary index search to the SPLIT operator.
+
+        // Fetches the conditional split variable from the secondary-index search
+        condSplitVars = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset, recordType, metaRecordType, inputOp,
+                secondaryIndex, SecondaryUnnestMapOutputVarType.CONDITIONAL_SPLIT_VAR);
+
+        // Adds a SPLIT operator after the given secondary index-search unnest-map operator.
+        splitOp = new SplitOperator(2,
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(condSplitVars.get(0))));
+        splitOp.getInputs().add(new MutableObject<ILogicalOperator>(currentOp));
+        splitOp.setExecutionMode(ExecutionMode.PARTITIONED);
+        context.computeAndSetTypeEnvironmentForOperator(splitOp);
+
+        // To maintain SSA, we assign new variables for the incoming variables in the left branch
+        // since the most tuples go to the right branch (instantTryLock success path). Also, the output of
+        // UNIONALL should be a new variable. (it cannot be the same to the left or right variable.)
+
+        // Original variables (before SPLIT) to the variables in the left path mapping
+        LinkedHashMap<LogicalVariable, LogicalVariable> liveVarAfterSplitToLeftPathMap = new LinkedHashMap<>();
+        // output variables to the variables generated in the left branch mapping
+        LinkedHashMap<LogicalVariable, LogicalVariable> origPKRecAndSKVarToleftPathMap = new LinkedHashMap<>();
+        // Original variables (before SPLIT) to the output variables mapping (mainly for join case)
+        LinkedHashMap<LogicalVariable, LogicalVariable> origVarToOutputVarMap = new LinkedHashMap<>();
+        List<LogicalVariable> liveVarsAfterSplitOp = new ArrayList<>();
+        VariableUtilities.getLiveVariables(splitOp, liveVarsAfterSplitOp);
+
+        ArrayList<LogicalVariable> assignVars = new ArrayList<>();
+        ArrayList<Mutable<ILogicalExpression>> assignExprs = new ArrayList<>();
+        for (LogicalVariable v : liveVarsAfterSplitOp) {
+            LogicalVariable newVar = context.newVar();
+            liveVarAfterSplitToLeftPathMap.put(v, newVar);
+            assignVars.add(newVar);
+            assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+        }
+        AssignOperator origVarsToLeftPathVarsAssignOp = new AssignOperator(assignVars, assignExprs);
+        origVarsToLeftPathVarsAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(splitOp));
+        context.computeAndSetTypeEnvironmentForOperator(origVarsToLeftPathVarsAssignOp);
+        origVarsToLeftPathVarsAssignOp.setExecutionMode(ExecutionMode.PARTITIONED);
+
+        // Creates the variable mapping for the UNIONALL operator.
+
+        // PK Variable(s) that will be fed into the primary index-search has been re-assigned in the left path.
+        List<LogicalVariable> pkVarsInLeftPathFromSIdxSearchBeforeSplit = new ArrayList<>();
+        for (LogicalVariable v : pkVarsFromSIdxUnnestMapOp) {
+            pkVarsInLeftPathFromSIdxSearchBeforeSplit.add(liveVarAfterSplitToLeftPathMap.get(v));
+        }
+        // PK and Record variable(s) from the primary-index search will be reassigned in the left path
+        // to make the output of the UNIONALL the original variables from the data-scan.
+        List<LogicalVariable> pkVarsFromPIdxSearchInLeftPath = new ArrayList<>();
+        for (int i = 0; i < primaryIndexUnnestVars.size(); i++) {
+            LogicalVariable replacedVar = context.newVar();
+            pkVarsFromPIdxSearchInLeftPath.add(replacedVar);
+            origPKRecAndSKVarToleftPathMap.put(primaryIndexUnnestVars.get(i), replacedVar);
+        }
+
+        // Are the used variables after SELECT or JOIN operator from the primary index?
+        // Then, creates the variable mapping between two paths.
+        for (LogicalVariable tVar : usedVarsAfterTopOp) {
+            // Checks whether this variable is already added to the union variable map.
+            // It should be also a part of the primary key variables.
+            if (findVarInTripleVarList(unionVarMap, tVar, false) || !primaryIndexUnnestVars.contains(tVar)) {
+                continue;
+            }
+            int pIndexPKIdx = primaryIndexUnnestVars.indexOf(tVar);
+            // If the above value is -1, either it is a secondary key variable or a variable
+            // from different branch (join case). These cases will be dealt with later.
+            if (pIndexPKIdx == -1) {
+                continue;
+            }
+            unionVarMap.add(new Triple<>(pkVarsFromPIdxSearchInLeftPath.get(pIndexPKIdx),
+                    pkVarsFromSIdxUnnestMapOp.get(pIndexPKIdx), tVar));
+            origVarToOutputVarMap.put(pkVarsFromSIdxUnnestMapOp.get(pIndexPKIdx), tVar);
+
+            // Constructs the mapping between the PK from the original data-scan to the PK
+            // from the secondary index search since they are different logical variables.
+            origVarToSIdxUnnestMapOpVarMap.put(tVar, pkVarsFromSIdxUnnestMapOp.get(pIndexPKIdx));
+        }
+
+        // Are the used variables after SELECT or JOIN operator from the given secondary index?
+        for (LogicalVariable tVar : usedVarsAfterTopOp) {
+            // Checks whether this variable is already added to the union variable map.
+            if (findVarInTripleVarList(unionVarMap, tVar, false)) {
+                continue;
+            }
+            // Should be either used in the condition or a composite index field that is not used in the condition.
+            if (!usedVarsInTopOp.contains(tVar) && !producedVarsInAssignsBeforeTopOp.contains(tVar)) {
+                continue;
+            }
+            int sIndexIdx = chosenIndexFieldNames.indexOf(subTree.getVarsToFieldNameMap().get(tVar));
+            // For the join-case, the match might not exist.
+            // In this case, we just propagate the variables later.
+            if (sIndexIdx == -1) {
+                continue;
+            }
+            if (idxType == IndexType.RTREE) {
+                // R-Tree case: we need this variable in case if we need to do an additional verification,
+                // or the secondary key field is used after SELECT or JOIN operator.
+                // We need to use the re-constructed secondary key from the result (an MBR) of R-Tree search.
+                // For the join case, the match might not exist.
+                // In this case, we just propagate the variables later.
+                if (!skFieldUsedAfterTopOp && !requireVerificationAfterSIdxSearch) {
+                    continue;
+                }
+                LogicalVariable replacedVar = context.newVar();
+                origPKRecAndSKVarToleftPathMap.put(tVar, replacedVar);
+                origSKFieldVarToNewSKFieldVarMap.put(tVar, restoredSKVarFromRTree.get(sIndexIdx));
+                unionVarMap.add(new Triple<>(replacedVar, restoredSKVarFromRTree.get(sIndexIdx), tVar));
+                continue;
+            }
+            // B-Tree case:
+            LogicalVariable replacedVar = context.newVar();
+            origPKRecAndSKVarToleftPathMap.put(tVar, replacedVar);
+            origVarToOutputVarMap.put(skVarsFromSIdxUnnestMap.get(sIndexIdx), tVar);
+            unionVarMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(replacedVar,
+                    skVarsFromSIdxUnnestMap.get(sIndexIdx), tVar));
+            // Constructs the mapping between the SK from the original data-scan
+            // and the SK from the secondary index search since they are different logical variables.
+            origVarToSIdxUnnestMapOpVarMap.put(tVar, skVarsFromSIdxUnnestMap.get(sIndexIdx));
+        }
+
+        // For R-Tree case: if the given secondary key field variable is used only in the select or join condition,
+        // we were not able to catch the mapping between the original secondary key field and the newly restored
+        // secondary key field in the assign operator in the right path.
+        if (idxType == IndexType.RTREE && (skFieldUsedAfterTopOp || requireVerificationAfterSIdxSearch)) {
+            for (LogicalVariable v : uniqueUsedVarsInTopOp) {
+                if (!primaryIndexUnnestVars.contains(v)) {
+                    origSKFieldVarToNewSKFieldVarMap.put(v, restoredSKVarFromRTree.get(0));
+                }
+            }
+        }
+
+        // For the index-nested-loop join case,
+        // we propagate all variables that come from the outer relation and are used after join operator.
+        // Adds the variables that are both live after JOIN and used after the JOIN operator.
+        VariableUtilities.getLiveVariables((ILogicalOperator) topOpRef.getValue(), liveVarsAfterTopOp);
+        for (LogicalVariable v : usedVarsAfterTopOp) {
+            if (!liveVarsAfterTopOp.contains(v) || findVarInTripleVarList(unionVarMap, v, false)) {
+                continue;
+            }
+            LogicalVariable outputVar = context.newVar();
+            origVarToOutputVarMap.put(v, outputVar);
+            unionVarMap.add(new Triple<>(liveVarAfterSplitToLeftPathMap.get(v), v, outputVar));
+        }
+
+        // Replaces the original variables in the operators after the SELECT or JOIN operator to satisfy SSA.
+        if (afterTopOpRefs != null) {
+            for (Mutable<ILogicalOperator> afterTopOpRef : afterTopOpRefs) {
+                VariableUtilities.substituteVariables((ILogicalOperator) afterTopOpRef.getValue(),
+                        origVarToOutputVarMap, context);
+            }
+        }
+
+        // Creates the primary index lookup operator.
+        // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
+                retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
+                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes);
+        primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
+        context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
+        primaryIndexUnnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
+
+        // Now, generates the UnionAllOperator to merge the left and right paths.
+        // If we are transforming a join, in the instantTryLock on PK fail path, a SELECT operator should be
+        // constructed from the join condition and placed after the primary index lookup
+        // to do the final verification. If this is a select plan, we just need to use the original
+        // SELECT operator after the primary index lookup to do the final verification.
+        LinkedHashMap<LogicalVariable, LogicalVariable> origVarToNewVarInLeftPathMap = new LinkedHashMap<>();
+        origVarToNewVarInLeftPathMap.putAll(liveVarAfterSplitToLeftPathMap);
+        origVarToNewVarInLeftPathMap.putAll(origPKRecAndSKVarToleftPathMap);
+        ILogicalExpression conditionRefExpr = conditionRef.getValue().cloneExpression();
+        // The retainMissing variable contains the information whether we are optimizing a left-outer join or not.
+        LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
+        newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
+                retainMissing, newMissingPlaceHolderVar);
+        VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
+
+        // If there are ASSIGN operators before the SELECT or JOIN operator,
+        // we need to put these operators between the SELECT or JOIN and the primary index lookup in the left path.
+        if (assignsBeforeTopOpRef != null) {
+            // Makes the primary unnest-map as the child of the last ASSIGN (from top) in the path.
+            assignBeforeTopOp = assignsBeforeTopOpRef.get(assignsBeforeTopOpRef.size() - 1).getValue();
+            assignBeforeTopOp.getInputs().clear();
+            assignBeforeTopOp.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestMapOp));
+
+            // Makes the first ASSIGN (from top) as the child of the SELECT operator.
+            for (int i = assignsBeforeTopOpRef.size() - 1; i >= 0; i--) {
+                if (assignsBeforeTopOpRef.get(i) != null) {
+                    AbstractLogicalOperator assignTmpOp =
+                            (AbstractLogicalOperator) assignsBeforeTopOpRef.get(i).getValue();
+                    assignTmpOp.setExecutionMode(ExecutionMode.PARTITIONED);
+                    VariableUtilities.substituteVariables(assignTmpOp, origVarToNewVarInLeftPathMap, context);
+                    context.computeAndSetTypeEnvironmentForOperator(assignTmpOp);
+                }
+            }
+            newSelectOpInLeftPath.getInputs().clear();
+            newSelectOpInLeftPath.getInputs()
+                    .add(new MutableObject<ILogicalOperator>(assignsBeforeTopOpRef.get(0).getValue()));
+        } else {
+            newSelectOpInLeftPath.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestMapOp));
+        }
+        newSelectOpInLeftPath.setExecutionMode(ExecutionMode.PARTITIONED);
+        context.computeAndSetTypeEnvironmentForOperator(newSelectOpInLeftPath);
+
+        // Now, we take care of the right path (instantTryLock on PK success path).
+        ILogicalOperator currentTopOpInRightPath = splitOp;
+        // For an R-Tree index, if there are operators that are using the secondary key field value,
+        // we need to reconstruct that field value from the result of R-Tree search.
+        // This is done by adding the following assign operator that we have made in the beginning of this method.
+        if (skVarAssignOpInRightPath != null) {
+            skVarAssignOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(splitOp));
+            skVarAssignOpInRightPath.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(skVarAssignOpInRightPath);
+            currentTopOpInRightPath = skVarAssignOpInRightPath;
+        }
+
+        // For an R-Tree index, if the given query shape is not RECTANGLE or POINT,
+        // we need to add the original SELECT operator to filter out the false positive results.
+        // (e.g., spatial-intersect($o.pointfield, create-circle(create-point(30.0,70.0), 5.0)) )
+        //
+        // Also, for a B-Tree composite index, we need to apply SELECT operators in the right path
+        // to remove any false positive results from the secondary composite index search.
+        //
+        // Lastly, if there is an index-nested-loop-join and the join contains more conditions
+        // other than joining fields, then those conditions need to be applied to filter out
+        // false positive results in the right path.
+        // (e.g., where $a.authors /*+ indexnl */ = $b.authors and $a.id = $b.id   <- authors:SK, id:PK)
+        if ((idxType == IndexType.RTREE || uniqueUsedVarsInTopOp.size() > 1) && requireVerificationAfterSIdxSearch) {
+            // Creates a new SELECT operator by deep-copying the SELECT operator in the left path
+            // since we need to change the variable reference in the SELECT operator.
+            // For the index-nested-loop join case, we copy the condition of the join operator.
+            ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
+            newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
+                    retainMissing, newMissingPlaceHolderVar);
+            newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
+            VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
+            VariableUtilities.substituteVariables(newSelectOpInRightPath, origSKFieldVarToNewSKFieldVarMap, context);
+            newSelectOpInRightPath.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(newSelectOpInRightPath);
+            currentTopOpInRightPath = newSelectOpInRightPath;
+        }
+
+        // Adds the new missing place holder in case of a left-outer-join if it's not been added yet.
+        // The assumption here is that this variable is the first PK variable that was set.
+        if (retainMissing && newMissingPlaceHolderForLOJ == primaryIndexUnnestVars.get(0)
+                && !findVarInTripleVarList(unionVarMap, newMissingPlaceHolderForLOJ, false)) {
+            unionVarMap.add(new Triple<>(origPKRecAndSKVarToleftPathMap.get(newMissingPlaceHolderForLOJ),
+                    pkVarsFromSIdxUnnestMapOp.get(0), newMissingPlaceHolderForLOJ));
+        }
+
+        // UNIONALL operator that combines both paths.
+        unionAllOp = new UnionAllOperator(unionVarMap);
+        unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(newSelectOpInLeftPath));
+        unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
+
+        unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
+        context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
+
+        // If an assign operator that keeps constant values was added, set the UNIONALL operator as its child.
+        if (!constAssignVars.isEmpty() && !constantAssignVarUsedInTopOp) {
+            constAssignOp.getInputs().clear();
+            constAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(unionAllOp));
+            constAssignOp.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(constAssignOp);
+
+            // This constant assign operator is the new child of the first operator after the original
+            // SELECT or JOIN operator.
+            currentOpAfterTopOp = afterTopOpRefs.get(afterTopOpRefs.size() - 1).getValue();
+            currentOpAfterTopOp.getInputs().clear();
+            currentOpAfterTopOp.getInputs().add(new MutableObject<ILogicalOperator>(constAssignOp));
+            context.computeAndSetTypeEnvironmentForOperator(currentOpAfterTopOp);
+            afterTopOpRefs.add(new MutableObject<ILogicalOperator>(constAssignOp));
+        }
+
+        // Index-only plan is now constructed. Return this operator to the caller.
+        return unionAllOp;
+    }
+
+    private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
+            boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
+            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
+            throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -570,22 +1336,17 @@ public class AccessMethodUtils {
         jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
         jobGenParams.setIsEqCondition(true);
         jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
-        // Variables and types coming out of the primary-index search.
-        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
-        List<Object> primaryIndexOutputTypes = new ArrayList<>();
-        // Append output variables/types generated by the primary-index search (not forwarded from input).
-        primaryIndexUnnestVars.addAll(dataSourceOp.getVariables());
-        appendPrimaryIndexTypes(dataset, recordType, metaRecordType, primaryIndexOutputTypes);
         // An index search is expressed as an unnest over an index-search function.
         IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         AbstractFunctionCallExpression primaryIndexSearchFunc =
                 new ScalarFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
-        // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
-        // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
-        AbstractUnnestMapOperator primaryIndexUnnestOp = null;
-        if (retainNull) {
+        // This is the operator that jobgen will be looking for. It contains an unnest function that has
+        // all necessary arguments to determine which index to use, which variables contain the index-search keys,
+        // what is the original dataset, etc.
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp = null;
+        if (retainMissing) {
             if (retainInput) {
-                primaryIndexUnnestOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
+                primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
                         new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                         retainInput);
             } else {
@@ -593,29 +1354,98 @@ public class AccessMethodUtils {
                 throw new AlgebricksException("Left-outer-join should propagate all inputs from the outer branch.");
             }
         } else {
-            primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
+            primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                     retainInput);
         }
-        // Fed by the order operator or the secondaryIndexUnnestOp.
-        if (sortPrimaryKeys) {
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+        return primaryIndexUnnestMapOp;
+    }
+
+    /**
+     * Creates operators that do a primary index lookup in the plan. In case of an index-only plan,
+     * this creates two paths including the primary index lookup in the left path.
+     * If this is an index-only plan (only using PK and/or secondary field(s) after SELECT operator) and/or
+     * the combination of the SELECT (JOIN) condition and the chosen secondary index do not generate
+     * false positive results, we can apply instantTryLock() on PK optimization since a result from these indexes
+     * doesn't have to be verified by the primary index-lookup and a subsequent SELECT operator.
+     * (i.e., we can guarantee the correctness of the result.)
+     *
+     * Case A) non-index-only plan
+     * sidx-search -> (optional) sort -> pdix-search
+     *
+     * Case B) index-only plan
+     * left path (an instantTryLock() on the PK fail path):
+     * right path(an instantTryLock() on the PK success path):
+     * (left) sidx-search -> assign? -> split -> primary index-search -> select (verification) -> union ->
+     * (right) ........................ split -> assign? -> select? -> .......................... union ...
+     */
+    public static ILogicalOperator createRestOfIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignsBeforeTopOpRef, AbstractDataSourceOperator dataSourceOp,
+            Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
+            IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
+            boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
+            OptimizableOperatorSubTree subTree, LogicalVariable newMissingPlaceHolderForLOJ)
+            throws AlgebricksException {
+        // Common part for the non-index-only plan and index-only plan
+        // Variables and types for the primary-index search.
+        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
+        List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+        // Appends output variables/types generated by the primary-index search (not forwarded from input).
+        primaryIndexUnnestVars.addAll(dataSourceOp.getVariables());
+        appendPrimaryIndexTypes(dataset, recordType, metaRecordType, primaryIndexOutputTypes);
+
+        // Fetches PK variable(s) from the secondary-index search operator.
+        List<LogicalVariable> pkVarsFromSIdxUnnestMapOp = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset,
+                recordType, metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
+
+        // Index-only plan or not?
+        boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
+
+        // Non-index-only plan case: creates ORDER -> UNNEST-MAP(Primary-index search) and return that unnest-map op.
+        if (!isIndexOnlyPlan) {
+            return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, sortPrimaryKeys, retainInput,
+                    retainMissing, requiresBroadcast, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars,
+                    primaryIndexOutputTypes);
         } else {
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
+            // Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
+            // and returns it.
+            return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
+                    dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
+                    requiresBroadcast, secondaryIndex, analysisCtx, subTree, newMissingPlaceHolderForLOJ,
+                    pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
         }
-        context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
-        primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        return primaryIndexUnnestOp;
+    }
+
+    private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars) {
+        List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
+        AbstractFunctionCallExpression createPointExpr1 =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_POINT));
+        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(0))));
+        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(1))));
+        createPointExpr1.getArguments().addAll(expressions);
+        return createPointExpr1;
+    }
+
+    private static AbstractFunctionCallExpression createRectangleExpression(
+            AbstractFunctionCallExpression createPointExpr1, AbstractFunctionCallExpression createPointExpr2) {
+        List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
+        AbstractFunctionCallExpression createRectangleExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_RECTANGLE));
+        expressions.add(new MutableObject<ILogicalExpression>(createPointExpr1));
+        expressions.add(new MutableObject<ILogicalExpression>(createPointExpr2));
+        createRectangleExpr.getArguments().addAll(expressions);
+        return createRectangleExpr;
     }
 
     public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp)
             throws AlgebricksException {
-        //find IS_NULL function of which argument has the nullPlaceholder variable in the nested plan of groupby.
+        //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
         ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
         Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
         AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
-        boolean foundSelectNonNull = false;
-        ScalarFunctionCallExpression isNullFuncExpr = null;
+        boolean foundSelectNonMissing = false;
+        ScalarFunctionCallExpression isMissingFuncExpr = null;
         AbstractLogicalOperator inputOp = subPlanRootOp;
         while (inputOp != null) {
             if (inputOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
@@ -629,11 +1459,11 @@ public class AccessMethodUtils {
                                 .getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                             if (((AbstractFunctionCallExpression) notFuncExpr.getArguments().get(0).getValue())
                                     .getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
-                                isNullFuncExpr =
+                                isMissingFuncExpr =
                                         (ScalarFunctionCallExpression) notFuncExpr.getArguments().get(0).getValue();
-                                if (isNullFuncExpr.getArguments().get(0).getValue()
+                                if (isMissingFuncExpr.getArguments().get(0).getValue()
                                         .getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                    foundSelectNonNull = true;
+                                    foundSelectNonMissing = true;
                                     break;
                                 }
                             }
@@ -645,21 +1475,20 @@ public class AccessMethodUtils {
                     : null;
         }
 
-        if (!foundSelectNonNull) {
-            throw new AlgebricksException(
-                    "Could not find the non-null select operator in GroupByOperator for LEFTOUTERJOIN plan optimization.");
+        if (!foundSelectNonMissing) {
+            throw CompilationException.create(ErrorCode.CANNOT_FIND_NON_MISSING_SELECT_OPERATOR);
         }
-        return isNullFuncExpr;
+        return isMissingFuncExpr;
     }
 
-    public static void resetLOJNullPlaceholderVariableInGroupByOp(AccessMethodAnalysisContext analysisCtx,
-            LogicalVariable newNullPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException {
+    public static void resetLOJMissingPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
+            LogicalVariable newMissingPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException {
 
-        //reset the null placeholder variable in groupby operator
-        ScalarFunctionCallExpression isNullFuncExpr = analysisCtx.getLOJIsNullFuncInGroupBy();
-        isNullFuncExpr.getArguments().clear();
-        isNullFuncExpr.getArguments().add(
-                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newNullPlaceholderVaraible)));
+        //reset the missing placeholder variable in groupby operator
+        ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInGroupBy();
+        isMissingFuncExpr.getArguments().clear();
+        isMissingFuncExpr.getArguments().add(
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newMissingPlaceholderVaraible)));
 
         //recompute type environment.
         OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJGroupbyOpRef(), context);
@@ -695,11 +1524,11 @@ public class AccessMethodUtils {
     }
 
     public static UnnestMapOperator createExternalDataLookupUnnestMap(AbstractDataSourceOperator dataSourceOp,
-            Dataset dataset, ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context,
-            boolean retainInput, boolean retainNull) throws AlgebricksException {
-        List<LogicalVariable> primaryKeyVars =
-                AccessMethodUtils.getPrimaryKeyVarsFromSecondaryUnnestMap(dataset, inputOp);
-
+            Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
+            IOptimizationContext context, Index secondaryIndex, boolean retainInput, boolean retainNull)
+            throws AlgebricksException {
+        List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset, recordType,
+                metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
         // add a sort on the RID fields before fetching external data.
         OrderOperator order = new OrderOperator();
         for (LogicalVariable pkVar : primaryKeyVars) {
@@ -774,4 +1603,845 @@ public class AccessMethodUtils {
         return usedVars.isEmpty() ? false : true;
     }
 
+    /**
+     * Checks whether the given function can generate false-positive results when using a corresponding index type.
+     */
+    public static Pair<Boolean, Boolean> canFunctionGenerateFalsePositiveResultsUsingIndex(
+            AbstractFunctionCallExpression funcExpr, List<Pair<FunctionIdentifier, Boolean>> funcIdents) {
+        boolean requireVerificationAfterSIdxSearch = true;
+
+        // Check whether the given function-call can generate false positive results.
+        FunctionIdentifier argFuncIdent = funcExpr.getFunctionIdentifier();
+        boolean functionFound = false;
+        for (int i = 0; i < funcIdents.size(); i++) {
+            if (argFuncIdent.equals(funcIdents.get(i).first)) {
+                functionFound = true;
+                requireVerificationAfterSIdxSearch = funcIdents.get(i).second;
+                break;
+            }
+        }
+
+        // If function-call itself is not an index-based access method, we check its arguments.
+        if (!functionFound) {
+            for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                ILogicalExpression argExpr = arg.getValue();
+                if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                    continue;
+                }
+                AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
+                FunctionIdentifier argExprFuncIdent = argFuncExpr.getFunctionIdentifier();
+                for (int i = 0; i < funcIdents.size(); i++) {
+                    if (argExprFuncIdent.equals(funcIdents.get(i).first)) {
+                        functionFound = true;
+                        requireVerificationAfterSIdxSearch = funcIdents.get(i).second;
+                        break;
+                    }
+                }
+            }
+        }
+
+        return new Pair<>(functionFound, requireVerificationAfterSIdxSearch);
+    }
+
+    /**
+     * Checks whether the given plan is an index-only plan (a.k.a. instantTryLock() on PK optimization).
+     * Refer to the IntroduceSelectAccessMethodRule or IntroduceJoinAccessMethodRule for more details.
+     *
+     * @throws AlgebricksException
+     */
+    public static void indexOnlyPlanCheck(List<Mutable<ILogicalOperator>> afterTopRefs,
+            Mutable<ILogicalOperator> topRef, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo)
+            throws AlgebricksException {
+        // First, checks all cases that the index-only can't be applied. If so, we can stop here.
+        Dataset dataset = indexSubTree.getDataset();
+        // For an external dataset, primary index, we don't apply index-only plan optimization.
+        // For a non-enforced index, we also don't apply index-only plan since it can contain a casted numeric value.
+        // For an enforced index, we also don't apply the index-only pan since the key field from a secondary index
+        // may not be equal to the actual value in the record. (e.g., INT index and BIGINT value in the actual record)
+        // Since index-only plan doesn't access the primary index, we can't get the actual value in this case.
+        // Also, if no-index-only option is given, we stop here to honor that request.
+        boolean noIndexOnlyPlanOption = getNoIndexOnlyOption(context);
+        // TODO: For the inverted index access-method cases only:
+        // Since an inverted index can contain multiple secondary key entries per one primary key,
+        // Index-only plan can't be applied. For example, suppose there are two entries (SK1, SK2) for one PK.
+        // Since we can't access <SK1, PK>, <SK2, PK> at the same time unless we use tryLock (we use instantTryLock),
+        // right now, we can't support an index-only plan on an inverted index.
+        // Once this issue is resolved, we can apply an index-only plan.
+        // One additional condition:
+        // Even if the above is resolved, if a secondary key field is used after
+        // SELECT or JOIN operator, this can't be qualified as an index-only plan since
+        // an inverted index contains a part of a field value, not all of it.
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL || chosenIndex.isPrimaryIndex()
+                || chosenIndex.isOverridingKeyFieldTypes() || chosenIndex.isEnforced() || isInvertedIndex(chosenIndex)
+                || noIndexOnlyPlanOption) {
+            indexOnlyPlanInfo.setFirst(false);
+            return;
+        }
+
+        // index-only plan possible?
+        boolean isIndexOnlyPlan = false;
+
+        // secondary key field usage after the select (join) operators
+        // This boolean is mainly used for R-Tree case since R-Tree index generates an MBR
+        // and we can restore original point or rectangle from this MBR if an index is built on point or rectangle.
+        boolean secondaryKeyFieldUsedAfterSelectOrJoinOp;
+
+        // Whether a post verification (especially for R-Tree case) is required after the secondary index search
+        // (e.g., the shape of the given query is not a point or rectangle.
+        //        Then, we may need to apply the select again using the real polygon, not MBR of it to get the true
+        //        result, not a super-set of it.)
+        boolean requireVerificationAfterSIdxSearch = indexOnlyPlanInfo.getThird();
+
+        // Does the given index can cover all search predicates?
+        boolean doesSIdxSearchCoverAllPredicates;
+
+        // matched function expressions
+        List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.getMatchedFuncExprs();
+
+        // logical variables that select (join) operator is using
+        List<LogicalVariable> usedVarsInSelJoinOp = new ArrayList<>();
+        List<LogicalVariable> usedVarsInSelJoinOpTemp = new ArrayList<>();
+
+        // live variables that select (join) operator can access
+        List<LogicalVariable> liveVarsAfterSelJoinOp = new ArrayList<>();
+
+        // PK, record variable
+        List<LogicalVariable> dataScanPKRecordVars;
+        List<LogicalVariable> dataScanPKVars = new ArrayList<>();
+        List<LogicalVariable> dataScanRecordVars = new ArrayList<>();
+
+        // Collects the used variables in the given select (join) operator.
+        VariableUtilities.getUsedVariables((ILogicalOperator) topRef.getValue(), usedVarsInSelJoinOpTemp);
+
+        // Removes the duplicated variables that are used in the select (join) operator
+        // in case where the variable is used multiple times in the operator's expression.
+        // (e.g., $i < 100 and $i > 10)
+        copyVarsToAnotherList(usedVarsInSelJoinOpTemp, usedVarsInSelJoinOp);
+
+        // If this is a join, we need to traverse the index subtree and find possible SELECT conditions
+        // since there may be more SELECT conditions and we need to collect used variables.
+        List<LogicalVariable> selectInIndexSubTreeVars = new ArrayList<>();
+        if (probeSubTree != null) {
+            ILogicalOperator tmpOp = indexSubTree.getRoot();
+            while (tmpOp.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+                if (tmpOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                    VariableUtilities.getUsedVariables(tmpOp, selectInIndexSubTreeVars);
+                    // Remove any duplicated variables.
+                    copyVarsToAnotherList(selectInIndexSubTreeVars, usedVarsInSelJoinOp);
+                    selectInIndexSubTreeVars.clear();
+                }
+                tmpOp = tmpOp.getInputs().get(0).getValue();
+            }
+        }
+        usedVarsInSelJoinOpTemp.clear();
+
+        // For the index-nested-loop join case, we need to remove variables from the left (outer) relat

<TRUNCATED>

[09/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1019-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1019-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1019-index-only.plan
new file mode 100644
index 0000000..6a298c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1019-index-only.plan
@@ -0,0 +1,145 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$39][$$54]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$98, $$92]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$98(ASC), $$92(ASC)]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$98, $$92]  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- SPLIT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- RTREE_SEARCH  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- PRE_CLUSTERED_GROUP_BY[$$40, $$52]  |PARTITIONED|
+                                                                                                    {
+                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                        -- STREAM_SELECT  |LOCAL|
+                                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                    }
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STABLE_SORT [$$40(ASC), $$52(ASC)]  |PARTITIONED|
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$40, $$52]  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- HYBRID_HASH_JOIN [$$50][$$43]  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
+                                                                                                            -- NESTED_LOOP  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SPLIT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- RTREE_SEARCH  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$40, $$52]  |PARTITIONED|
+                                                                                              {
+                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                              }
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STABLE_SORT [$$40(ASC), $$52(ASC)]  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$40, $$52]  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$50][$$43]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
+                                                                                                      -- NESTED_LOOP  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.aql
new file mode 100644
index 0000000..bae1c67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.1.ddl.aql
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test filters with loading and in the existence of a secondary b-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int64,
+        author-id: int64,
+        in-response-to: int64?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.aql
new file mode 100644
index 0000000..28a0eb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.2.update.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="asterix_nc1://data/fbm-with-send-time.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.aql
new file mode 100644
index 0000000..265ddec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.3.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 dataverse test;
+
+create index fbAuthorIdx on FacebookMessages(author-id) type btree;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.4.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.aql
new file mode 100644
index 0000000..4aa01f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree-index-only/load-with-secondary-btree-index-only.5.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where $m.author-id >= 1
+order by $m.message-id
+return {"message-id":$m.message-id,"author-id":$m.author-id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..efa4aee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.aql
new file mode 100644
index 0000000..3bee139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.aql
@@ -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.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.aql
new file mode 100644
index 0000000..8ee1f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.aql
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+count(
+for $c in dataset('Customers')
+for $o in dataset('Orders')
+where $c.cid /*+ indexnl */ = $o.cid
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.aql
new file mode 100644
index 0000000..8170cc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.aql
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  This is an index-only plan since we are using the
+ *                  secondary key field condition and returning only PK and SK fields.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int64,
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.aql
new file mode 100644
index 0000000..f7c3926
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset DBLP
+using localfs
+(("path"="asterix_nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using localfs
+(("path"="asterix_nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.aql
new file mode 100644
index 0000000..a6c1c56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+create index authors_index on DBLP(authors);
+create index authors_index on CSX(authors);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.aql
new file mode 100644
index 0000000..cb11e57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors /*+ indexnl */ = $b.authors
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "title": $b.title}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..875820d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.aql
new file mode 100644
index 0000000..0605ade
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.aql
new file mode 100644
index 0000000..c6f1303
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 800 and $o.cid /*+ indexnl */ = $c.cid
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..476185a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
@@ -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.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Each branch (outer and inner) will be transformed as an index-only plan.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
new file mode 100644
index 0000000..461f5c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData2.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
new file mode 100644
index 0000000..2066c68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 100000 and $o.cid /*+ indexnl */ = $c.cashBack
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..baa89f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.aql
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ *                  This is not an index-only plan since a single secondary index can't
+ *                  cover the whole search predicate for the outer relation even when we exclude the join condition.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.aql
new file mode 100644
index 0000000..0605ade
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.aql
new file mode 100644
index 0000000..6294ef3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 800 and $o.total < 10000 and $o.cid /*+ indexnl */ = $c.cid
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..e5290ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Inner branch will be transformed as an index-only plan.
+ *                  Outer branch cannot be transformed as an index-only plan as an index can't cover
+ *                  all search predicates even excluding the join condition.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
new file mode 100644
index 0000000..461f5c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData2.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
new file mode 100644
index 0000000..510f439
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 100000 and $o.total >= 0 and $o.cid /*+ indexnl */ = $c.cashBack
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.aql
deleted file mode 100644
index b14d0ce..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int64,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle,
-  circle: circle
-}
-
-create dataset MyData1(MyRecord) primary key id;
-create dataset MyData2(MyRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.aql
deleted file mode 100644
index 6f43a5b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.aql
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-load dataset MyData1
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-
-load dataset MyData2
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.aql
deleted file mode 100644
index 6ecdc5e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.aql
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-create index rtree_index on MyData1(point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.aql
deleted file mode 100644
index ed7af43..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-for $a in dataset('MyData1')
-for $b in dataset('MyData2')
-where spatial-intersect($a.point, $b.point) and $a.id != $b.id
-order by $a.id, $b.id
-return {"aid": $a.id, "bid": $b.id, "apt": $a.point, "bp": $b.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.aql
new file mode 100644
index 0000000..b14d0ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.aql
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.aql
new file mode 100644
index 0000000..6f43a5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the 
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.aql
new file mode 100644
index 0000000..6ecdc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the 
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+create index rtree_index on MyData1(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.aql
new file mode 100644
index 0000000..ed7af43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the 
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('MyData1')
+for $b in dataset('MyData2')
+where spatial-intersect($a.point, $b.point) and $a.id != $b.id
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "apt": $a.point, "bp": $b.point}


[03/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.adm
new file mode 100644
index 0000000..f8b98e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.adm
@@ -0,0 +1,204 @@
+{ "aid": 1, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 1, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 1, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 1, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 2, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 2, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 2, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 2, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 3, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 3, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 3, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 3, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 4, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 4, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 4, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 4, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 6, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 6, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 6, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 6, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 7, "bid": 8, "apoly1": polygon("-5.0,-2.0 -4.0,-1.0 -3.0,-1.0 -2.0,-2.0 -4.0,-4.0 -5.0,-3.0"), "bpoly2": polygon("-3.0,-3.0 -1.0,-3.0 -3.0,-5.0") }
+{ "aid": 8, "bid": 8, "apoly1": polygon("-5.0,-2.0 -4.0,-1.0 -3.0,-1.0 -2.0,-2.0 -4.0,-4.0 -5.0,-3.0"), "bpoly2": polygon("-3.0,-3.0 -1.0,-3.0 -3.0,-5.0") }
+{ "aid": 9, "bid": 3, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 9, "bid": 4, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 9, "bid": 6, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 9, "bid": 7, "apoly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 1, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 2, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 5, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 6, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 7, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 9, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 10, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 11, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 12, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 13, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 14, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 15, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 16, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 17, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 10, "bid": 18, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 10, "bid": 19, "apoly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 11, "bid": 1, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 2, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 5, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 6, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 7, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 9, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 10, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 11, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 12, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 13, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 14, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 15, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 16, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 17, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 11, "bid": 18, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 11, "bid": 19, "apoly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 12, "bid": 1, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 2, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 5, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 6, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 7, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 9, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 10, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 11, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 12, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 13, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 14, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 15, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 16, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 17, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 12, "bid": 18, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 12, "bid": 19, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 13, "bid": 1, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 2, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 5, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 6, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 7, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 9, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 10, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 11, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 12, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 13, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 14, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 15, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 16, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 17, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 13, "bid": 18, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 13, "bid": 19, "apoly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 14, "bid": 1, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 2, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 5, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 6, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 7, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 9, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 10, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 11, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 12, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 13, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 14, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 15, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 16, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 17, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 14, "bid": 18, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 14, "bid": 19, "apoly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 15, "bid": 1, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 2, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 5, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 6, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 7, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 9, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 10, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 11, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 12, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 13, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 14, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 15, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 16, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 17, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 15, "bid": 18, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 15, "bid": 19, "apoly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 16, "bid": 1, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 2, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 5, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 6, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 7, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 9, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 10, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 11, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 12, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 13, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 14, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 15, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 16, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 17, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 16, "bid": 18, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 16, "bid": 19, "apoly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 17, "bid": 1, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 2, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 3, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0") }
+{ "aid": 17, "bid": 4, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0") }
+{ "aid": 17, "bid": 5, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 6, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 7, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 9, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 10, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 11, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 12, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 13, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 14, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 15, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 16, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 17, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 17, "bid": 18, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 17, "bid": 19, "apoly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 18, "bid": 1, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 2, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 5, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 6, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 7, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 9, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 10, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 11, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 12, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 13, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 14, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 15, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 16, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 17, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 18, "bid": 19, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 19, "bid": 1, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 2, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 5, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 6, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 7, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 9, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 10, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 11, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 12, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 13, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 14, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 15, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 16, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 17, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 19, "bid": 19, "apoly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 20, "bid": 1, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 2, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 5, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 6, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 7, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 9, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 10, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 11, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 12, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 13, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 14, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 15, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 16, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 17, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0") }
+{ "aid": 20, "bid": 18, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 20, "bid": 19, "apoly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 21, "bid": 18, "apoly1": polygon("5.1,5.1 14.0,14.0 22.0,14.0 22.0,10.0"), "bpoly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0") }
+{ "aid": 21, "bid": 19, "apoly1": polygon("5.1,5.1 14.0,14.0 22.0,14.0 22.0,10.0"), "bpoly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.adm
new file mode 100644
index 0000000..ec63514
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.adm
@@ -0,0 +1 @@
+9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-rtree-sidx3-indexonly-plan.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-rtree-sidx3-indexonly-plan.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-rtree-sidx3-indexonly-plan.1.adm
new file mode 100644
index 0000000..ec63514
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-rtree-sidx3-indexonly-plan.1.adm
@@ -0,0 +1 @@
+9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.adm
new file mode 100644
index 0000000..cb1a40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.adm
@@ -0,0 +1 @@
+250

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
new file mode 100644
index 0000000..cb1a40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
@@ -0,0 +1 @@
+250

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.adm
new file mode 100644
index 0000000..cb1a40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.adm
@@ -0,0 +1 @@
+250

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
new file mode 100644
index 0000000..cb1a40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.adm
@@ -0,0 +1 @@
+250

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.adm
new file mode 100644
index 0000000..dbbd081
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.adm
@@ -0,0 +1,2 @@
+{ "pk": 1, "sk": 7 }
+{ "pk": 2, "sk": 1 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.adm
new file mode 100644
index 0000000..77076cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.adm
@@ -0,0 +1,2 @@
+{ "pk": 1, "sk": 1 }
+{ "pk": 2, "sk": 2 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.adm
new file mode 100644
index 0000000..e2c20e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.adm
@@ -0,0 +1,2 @@
+{ "pk": 1, "sk": 1, "sk2": 7 }
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.adm
new file mode 100644
index 0000000..77076cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.adm
@@ -0,0 +1,2 @@
+{ "pk": 1, "sk": 1 }
+{ "pk": 2, "sk": 2 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.adm
new file mode 100644
index 0000000..7ac394b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.adm
@@ -0,0 +1,2 @@
+{ "pk": 1 }
+{ "pk": 2 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.adm
new file mode 100644
index 0000000..aaaa784
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.adm
@@ -0,0 +1,2 @@
+{ "sk": 1 }
+{ "sk": 2 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.adm
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.adm
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.10.adm
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.adm
new file mode 100644
index 0000000..ec917ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.11.adm
@@ -0,0 +1,30 @@
+{ "id": 1 }
+{ "id": 9 }
+{ "id": 10 }
+{ "id": 11 }
+{ "id": 12 }
+{ "id": 13 }
+{ "id": 14 }
+{ "id": 17 }
+{ "id": 20 }
+{ "id": 21 }
+{ "id": 22 }
+{ "id": 23 }
+{ "id": 24 }
+{ "id": 27 }
+{ "id": 30 }
+{ "id": 31 }
+{ "id": 39 }
+{ "id": 40 }
+{ "id": 41 }
+{ "id": 42 }
+{ "id": 43 }
+{ "id": 44 }
+{ "id": 47 }
+{ "id": 50 }
+{ "id": 51 }
+{ "id": 52 }
+{ "id": 53 }
+{ "id": 54 }
+{ "id": 57 }
+{ "id": 60 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.adm
new file mode 100644
index 0000000..78b4996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.3.adm
@@ -0,0 +1 @@
+258176

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.adm
new file mode 100644
index 0000000..df0850a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.4.adm
@@ -0,0 +1,6 @@
+{ "id": 10 }
+{ "id": 20 }
+{ "id": 30 }
+{ "id": 40 }
+{ "id": 50 }
+{ "id": 60 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.adm
new file mode 100644
index 0000000..c1dce74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.5.adm
@@ -0,0 +1,6 @@
+{ "point": point("2.0,3.0") }
+{ "point": point("4.0,3.0") }
+{ "point": point("4.0,3.0") }
+{ "point": point("2.0,3.0") }
+{ "point": point("4.0,3.0") }
+{ "point": point("4.0,3.0") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.adm
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.6.adm
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.adm
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.7.adm
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.adm
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.8.adm
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.adm
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-indexonly-plan-01/rtree-secondary-index-indexonly-plan-01.9.adm
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-01/rtree-secondary-index-non-indexonly-plan-01.1.adm
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-non-indexonly-plan-02/rtree-secondary-index-non-indexonly-plan-02.1.adm
@@ -0,0 +1 @@
+10

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/results/orderby_limit/orderby_limit_02/orderby_limit_02.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/orderby_limit/orderby_limit_02/orderby_limit_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/orderby_limit/orderby_limit_02/orderby_limit_02.1.adm
index e027c88..174a652 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/orderby_limit/orderby_limit_02/orderby_limit_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/orderby_limit/orderby_limit_02/orderby_limit_02.1.adm
@@ -1,5 +1,5 @@
 { "name": "Susan" }
 { "name": "Smith" }
 { "name": "Roger" }
-{ "name": "Kevin" }
-{ "name": "John" }
+{ "name": "Ravi" }
+{ "name": "Ramesh" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 769b8a2..2cfb321 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2671,6 +2671,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-join">
+      <compilation-unit name="btree-primary-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-primary-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
       <compilation-unit name="btree-secondary-equi-join_01">
         <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
       </compilation-unit>
@@ -2686,13 +2691,58 @@
       </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-secondary-indexonly-plan-to-primary-equi-join_01">
+        <output-dir compare="Text">btree-secondary-indexonly-plan-to-primary-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-non-indexonly-plan-to-primary-equi-join_01">
+        <output-dir compare="Text">btree-secondary-non-indexonly-plan-to-primary-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
       <compilation-unit name="btree-secondary-self-equi-join_01">
         <output-dir compare="Text">btree-secondary-self-equi-join_01</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-join">
-      <compilation-unit name="rtree-spatial-intersect-point">
-        <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+      <compilation-unit name="rtree-spatial-intersect-point_01">
+        <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_02">
+        <output-dir compare="Text">rtree-spatial-intersect-point_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_03">
+        <output-dir compare="Text">rtree-spatial-intersect-point_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_04">
+        <output-dir compare="Text">rtree-spatial-intersect-point_04</output-dir>
+      </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-join">
@@ -2723,6 +2773,26 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-01">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-02">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-03">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-index-indexonly-plan-01">
+        <output-dir compare="Text">btree-secondary-index-indexonly-plan-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>
@@ -2843,6 +2913,21 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-indexonly-plan-01">
+        <output-dir compare="Text">rtree-secondary-index-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-non-indexonly-plan-01">
+        <output-dir compare="Text">rtree-secondary-index-non-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-non-indexonly-plan-02">
+        <output-dir compare="Text">rtree-secondary-index-non-indexonly-plan-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="disjunctive-predicate-1">
         <output-dir compare="Text">disjunctive-predicate-1</output-dir>
       </compilation-unit>
@@ -6710,6 +6795,11 @@
   </test-group>
   <test-group name="index-leftouterjoin">
     <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-pidx-with-join-btree-pidx1</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>
@@ -6720,6 +6810,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx3-indexonly-plan">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx3-indexonly-plan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
       <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
         <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
       </compilation-unit>
@@ -6730,6 +6825,26 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-indexonly-plan-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-indexonly-plan-with-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan">
+        <output-dir compare="Text">probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan">
+        <output-dir compare="Text">probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan</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>
@@ -7018,6 +7133,11 @@
       </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-ngram">
         <output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
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 16744fb..961fb41 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3012,6 +3012,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-join">
+      <compilation-unit name="btree-primary-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-primary-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
       <compilation-unit name="btree-secondary-equi-join_01">
         <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
       </compilation-unit>
@@ -3027,8 +3032,53 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-join">
-      <compilation-unit name="rtree-spatial-intersect-point">
-        <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+      <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-secondary-indexonly-plan-to-primary-equi-join_01">
+        <output-dir compare="Text">btree-secondary-indexonly-plan-to-primary-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-non-indexonly-plan-to-primary-equi-join_01">
+        <output-dir compare="Text">btree-secondary-non-indexonly-plan-to-primary-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01">
+        <output-dir compare="Text">btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_01">
+        <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_02">
+        <output-dir compare="Text">rtree-spatial-intersect-point_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_03">
+        <output-dir compare="Text">rtree-spatial-intersect-point_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_04">
+        <output-dir compare="Text">rtree-spatial-intersect-point_04</output-dir>
+      </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-group>
@@ -3074,6 +3124,31 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-01">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-02">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-composite-index-indexonly-plan-03">
+        <output-dir compare="Text">btree-secondary-composite-index-indexonly-plan-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-index-indexonly-plan-01">
+        <output-dir compare="Text">btree-secondary-index-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-secondary-index-non-indexonly-plan-01">
+        <output-dir compare="Text">btree-secondary-index-non-indexonly-plan-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>
@@ -3189,6 +3264,21 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-indexonly-plan-01">
+        <output-dir compare="Text">rtree-secondary-index-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-non-indexonly-plan-01">
+        <output-dir compare="Text">rtree-secondary-index-non-indexonly-plan-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-non-indexonly-plan-02">
+        <output-dir compare="Text">rtree-secondary-index-non-indexonly-plan-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="disjunctive-predicate-1">
         <output-dir compare="Text">disjunctive-predicate-1</output-dir>
       </compilation-unit>
@@ -8825,6 +8915,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx3-indexonly-plan">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx3-indexonly-plan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
       <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
         <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
       </compilation-unit>
@@ -8844,6 +8939,28 @@
         <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-indexonly-plan-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-indexonly-plan-with-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan">
+        <output-dir compare="Text">probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan">
+        <output-dir compare="Text">probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan</output-dir>
+      </compilation-unit>
+    </test-case>
+
+
   </test-group>
   <test-group name="distinct">
     <test-case FilePath="distinct">
@@ -9154,6 +9271,11 @@
       </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-ngram">
         <output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 1fc0f54..1a83603 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -71,6 +71,7 @@ public class ErrorCode {
     public static final int INVALID_DURATION = 28;
     public static final int UNKNOWN_DURATION_UNIT = 29;
     public static final int QUERY_TIMEOUT = 30;
+    public static final int INVALID_TYPE_CASTING_MATH_FUNCTION = 31;
 
     public static final int INSTANTIATION_ERROR = 100;
 
@@ -140,6 +141,11 @@ public class ErrorCode {
     public static final int WITH_FIELD_MUST_CONTAIN_SUB_FIELD = 1061;
     public static final int MERGE_POLICY_PARAMETER_INVALID_TYPE = 1062;
     public static final int NO_DATAVERSE_WITH_NAME = 1063;
+    public static final int ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION = 1064;
+    public static final int CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE = 1065;
+    public static final int CANNOT_SERIALIZE_A_VALUE = 1066;
+    public static final int CANNOT_FIND_NON_MISSING_SELECT_OPERATOR = 1067;
+    public static final int CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE = 1068;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 8341a33..995b541 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -64,6 +64,7 @@
 28 = Invalid duration \"%1$s\"
 29 = Unknown duration unit %1$s
 30 = Query timed out and will be cancelled
+31 = Invalid type-casting math function: %1$s for converting %2$s to %3$s
 
 100 = Unable to instantiate class %1$s
 
@@ -126,6 +127,11 @@
 1061 = Field \"%1$s\" in the with clause must contain sub field \"%2$s\"
 1062 = Merge policy parameters cannot be of type %1$s
 1063 = There is no dataverse with name \"%1$s\"
+1064 = An error was occurred while converting type %1$s to type %2$s.
+1065 = There should be at least two applicable indexes.
+1066 = Cannot serialize a value.
+1067 = Cannot find a non-missing SELECT operator in GROUP operator for a left-outer-join plan optimization.
+1068 = Cannot get the conditional split variable for the given UNNESTMAP operator.
 
 # Feed Errors
 3001 = Illegal state.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_parameters.md
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_parameters.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_parameters.md
index 3976937..6ba6aba 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_parameters.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_parameters.md
@@ -93,3 +93,21 @@ If there is no user-provided suffix, "B" is the default suffix. See the followin
     FROM GleambookUsers u JOIN GleambookMessages m ON m.authorId = u.id;
 
 
+## <a id="Index_Only">Controlling Index-Only-Plan Parameter</a>
+By default, the system tries to build an index-only plan whenever utilizing a secondary index is possible.
+For example, if a SELECT or JOIN query can utilize an enforced B+Tree or R-Tree index on a field, the optimizer
+checks whether a secondary-index search alone can generate the result that the query asks for. It
+mainly checks two conditions: (1) predicates used in WHERE only uses the primary key field and/or secondary key field
+and (2) the result does not return any other fields. If these two conditions hold, it builds an index-only plan.
+Since an index-only plan only searches a secondary-index to answer a query, it is faster than
+a non-index-only plan that needs to search the primary index.
+However, this index-only plan can be turned off per query by setting the following parameter.
+
+*  **noindexonly**: if this is set to true, the index-only-plan will not be applied; the default value is false.
+
+##### Example
+
+    SET noindexonly 'true';
+
+    SELECT m.message AS message
+    FROM GleambookMessages m where m.message = " love product-b its shortcut-menu is awesome:)";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index d682484..ddab5bc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -117,7 +117,7 @@ public class DatasetDataSource extends DataSource {
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
-                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes);
+                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, false);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }


[06/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..e4e4497
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+};
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+};
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
new file mode 100644
index 0000000..ddfd26b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
@@ -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.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+use test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
new file mode 100644
index 0000000..4bf06f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-primary-to-secondary-indexonly-plan-equi-join_01/btree-primary-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ * Success        : Yes
+ */
+
+use test;
+
+select value count(*) from (
+select o.oid, c.cid
+from Customers c, Orders o
+where c.cid /*+ indexnl */ = o.cid) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.sqlpp
new file mode 100644
index 0000000..bb700a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.1.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  This is an index-only plan since we are using the
+ *                  secondary key field condition and returning only PK and SK fields.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+};
+
+create type CSXType as closed {
+  id: int64,
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.sqlpp
new file mode 100644
index 0000000..e082c6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset DBLP
+using localfs
+(("path"="asterix_nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using localfs
+(("path"="asterix_nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.ddl.sqlpp
new file mode 100644
index 0000000..30aa05e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.3.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.
+ */
+
+use test;
+
+create index authors_index on DBLP(authors);
+create index authors_index on CSX(authors);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.sqlpp
new file mode 100644
index 0000000..f1b2162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-equi-join_04/btree-secondary-equi-join_04.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select a.id as aid, b.id as bid, b.title
+from DBLP a, CSX b
+where a.authors /*+ indexnl */ = b.authors
+order by aid, bid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..c53da0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+};
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+};
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
new file mode 100644
index 0000000..89bc827
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
new file mode 100644
index 0000000..494434a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-primary-equi-join_01/btree-secondary-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select o.oid as oid, c.cid as cid
+from Orders o, Customers c
+where o.cid < 800 and o.cid /*+ indexnl */ = c.cid
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..9042f33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Each branch (outer and inner) will be transformed as an index-only plan.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+};
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+};
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
new file mode 100644
index 0000000..dc5951f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData2.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
new file mode 100644
index 0000000..8e9417a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select o.oid as oid, c.cid as cid
+from Orders o, Customers c
+where o.cid < 100000 and o.cid /*+ indexnl */ = c.cashBack
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..fdc6a6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.1.ddl.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ *                  This is not an index-only plan since a single secondary index can't
+ *                  cover the whole search predicate for the outer relation even when we exclude the join condition.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+};
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+};
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
new file mode 100644
index 0000000..89bc827
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
new file mode 100644
index 0000000..b1160bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-primary-equi-join_01/btree-secondary-non-indexonly-plan-to-primary-equi-join_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select o.oid as oid, c.cid as cid
+from Orders o, Customers c
+where o.cid < 800 and o.total < 10000 and o.cid /*+ indexnl */ = c.cid
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..3e1d2cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.1.ddl.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Inner branch will be transformed as an index-only plan.
+ *                  Outer branch cannot be transformed as an index-only plan as an index can't cover
+ *                  all search predicates even excluding the join condition.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+};
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+};
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
new file mode 100644
index 0000000..dc5951f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/nontagged/customerData2.json"),("format"="adm"));
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/nontagged/orderData.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
new file mode 100644
index 0000000..63c94d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01/btree-secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select o.oid as oid, c.cid as cid
+from Orders o, Customers c
+where o.cid < 100000 and o.total >= 0 and o.cid /*+ indexnl */ = c.cashBack
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.sqlpp
deleted file mode 100644
index bde7f36..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.1.ddl.sqlpp
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create  dataset MyData1(MyRecord) primary key id;
-
-create  dataset MyData2(MyRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.sqlpp
deleted file mode 100644
index 21b9b96..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.2.update.sqlpp
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use test;
-
-
-load  dataset MyData1 using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-
-load  dataset MyData2 using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.sqlpp
deleted file mode 100644
index 01943c7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.3.ddl.sqlpp
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use test;
-
-
-create  index rtree_index  on MyData1 (point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.sqlpp
deleted file mode 100644
index 0cf19a3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point/rtree-spatial-intersect-point.4.query.sqlpp
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData1' has an RTree index, and we expect the
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use test;
-
-
-select element {'aid':a.id,'bid':b.id,'apt':a.point,'bp':b.point}
-from  MyData1 as a,
-      MyData2 as b
-where (test.`spatial-intersect`(a.point,b.point) and (a.id != b.id))
-order by a.id,b.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.sqlpp
new file mode 100644
index 0000000..bde7f36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+  id : bigint,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle,
+  circle : circle
+};
+
+create  dataset MyData1(MyRecord) primary key id;
+
+create  dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.sqlpp
new file mode 100644
index 0000000..21b9b96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use test;
+
+
+load  dataset MyData1 using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
+
+load  dataset MyData2 using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.sqlpp
new file mode 100644
index 0000000..01943c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.3.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use test;
+
+
+create  index rtree_index  on MyData1 (point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.sqlpp
new file mode 100644
index 0000000..0cf19a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_01/rtree-spatial-intersect-point_01.4.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData1' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use test;
+
+
+select element {'aid':a.id,'bid':b.id,'apt':a.point,'bp':b.point}
+from  MyData1 as a,
+      MyData2 as b
+where (test.`spatial-intersect`(a.point,b.point) and (a.id != b.id))
+order by a.id,b.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.sqlpp
new file mode 100644
index 0000000..65cd32e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.sqlpp
new file mode 100644
index 0000000..9116663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.sqlpp
new file mode 100644
index 0000000..a9f8edc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use test;
+
+create index rtree_index on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.sqlpp
new file mode 100644
index 0000000..e43f3d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use test;
+
+select element {'aid':a.id,'bid':b.id,'apoint':a.point,'bpoint':b.point}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.point,b.point)
+order by a.id,b.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.sqlpp
new file mode 100644
index 0000000..e004040
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ /*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.sqlpp
new file mode 100644
index 0000000..9116663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.sqlpp
new file mode 100644
index 0000000..0595f2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.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.
+ */
+
+use test;
+
+create index rtree_index1 on MyData1(point) type rtree;
+create index rtree_index2 on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.sqlpp
new file mode 100644
index 0000000..a241f6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {'aid':a.id,'bid':b.id,'apoint':a.point,'bpoint':b.point}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.point,b.point)
+order by a.id,b.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.sqlpp
new file mode 100644
index 0000000..94d2c2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.sqlpp
new file mode 100644
index 0000000..9116663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.sqlpp
new file mode 100644
index 0000000..9506862
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.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.
+ */
+
+use test;
+
+create index rtree_index1 on MyData1(poly1) type rtree;
+create index rtree_index2 on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.sqlpp
new file mode 100644
index 0000000..b3d9221
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {'aid':a.id,'bid':b.id,'apoly1':a.poly1,'bpoint':b.point}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.poly1,b.point)
+order by a.id,b.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.sqlpp
new file mode 100644
index 0000000..94d2c2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+


[12/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-02.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-02.aql
new file mode 100644
index 0000000..77633fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-02.aql
@@ -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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-03.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-03.aql
new file mode 100644
index 0000000..dc271af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-03.aql
@@ -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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"sk":$o.docid}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-04.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-04.aql
new file mode 100644
index 0000000..b1297f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-04.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id, "sk":$o.docid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-05.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-05.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-05.aql
new file mode 100644
index 0000000..3d1d1fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-05.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-06.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-06.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-06.aql
new file mode 100644
index 0000000..3aea6ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-06.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"sk":$o.docid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-07.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-07.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-07.aql
new file mode 100644
index 0000000..06d89b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-07.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"rec":$o, "pk":$o.id}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-08.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-08.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-08.aql
new file mode 100644
index 0000000..0892b7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-08.aql
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return $o
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-09.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-09.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-09.aql
new file mode 100644
index 0000000..a0cdc2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-09.aql
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type testType as {
+   id: uuid,
+   sid: int32
+}
+
+create dataset testDataset(testType) primary key id autogenerated
+create index testIdx on testDataset(sid)
+
+for $y in [0,2,4,6,8]
+return { "quantile":$y,
+         "count": count(for $x in dataset testDataset
+                        where $x.sid /* +indexnl */ <= $y
+                        return $x.sid)
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
new file mode 100644
index 0000000..b259612
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int32,
+    statuses-count: int32,
+    name: string,
+    followers-count: int32
+}
+
+create type TweetMessageNestedType as open {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int32,
+    countB: int32
+}
+
+create type TweetMessageType as open {
+    nested: TweetMessageNestedType
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key nested.tweetid;
+
+create index twmSndLocIx on TweetMessages(nested.sender-location) type rtree;
+create index msgCountAIx on TweetMessages(nested.countA) type btree;
+create index msgCountBIx on TweetMessages(nested.countB) type btree;
+create index msgTextIx on TweetMessages(nested.message-text) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+
+for $t1 in dataset('TweetMessages')
+let $n :=  create-circle($t1.nested.sender-location, 0.5)
+where $t1.nested.tweetid < int64("10")
+order by $t1.nested.tweetid
+return {
+"tweetid1": $t1.nested.tweetid,
+"loc1":$t1.nested.sender-location,
+"nearby-message": for $t2 in dataset('TweetMessages')
+                             where spatial-intersect($t2.nested.sender-location, $n)
+                             order by $t2.tweetid
+                             return {"tweetid2":$t2.nested.tweetid, "loc2":$t2.nested.sender-location}
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
index b259612..fbf8bd5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
@@ -59,7 +59,7 @@ create index msgCountAIx on TweetMessages(nested.countA) type btree;
 create index msgCountBIx on TweetMessages(nested.countB) type btree;
 create index msgTextIx on TweetMessages(nested.message-text) type keyword;
 
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 let $n :=  create-circle($t1.nested.sender-location, 0.5)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local-index-only.aql
new file mode 100644
index 0000000..bb0ffc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local-index-only.aql
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Time should remain partitoned
+ *                  : and be returned
+ *  Expected Result : Success
+ *  Date            : 20th Oct 2016
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int32,
+    statuses-count: int32,
+    name: string,
+    followers-count: int32
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int32,
+    countB: int32
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+
+for $t1 in dataset('TweetMessages')
+let $time := current-datetime()
+where $t1.tweetid < int64("10")
+order by $t1.tweetid
+return {
+"time": $time,
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+          where $t1.countA /* +indexnl */= $t2.countB
+          order by $t2.tweetid
+          return {"tweetid2": $t2.tweetid,
+                  "count2":$t2.countB}
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local.aql
index bb0ffc1..c0f490c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nonpure/keep-datetime-local.aql
@@ -56,7 +56,7 @@ create index msgCountAIx on TweetMessages(countA) type btree;
 create index msgCountBIx on TweetMessages(countB) type btree;
 create index msgTextIx on TweetMessages(message-text) type keyword;
 
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 let $time := current-datetime()

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search-index-only.aql
new file mode 100644
index 0000000..2815fc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search-index-only.aql
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop dataverse index_search if exists;
+create dataverse index_search;
+use dataverse index_search;
+
+create type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_Custkey_Orderstatus on Orders(o_custkey, o_orderstatus);
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40 and $o.o_orderstatus = "P"
+return {
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey,
+  "o_orderstatus": $o.o_orderstatus
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search.aql
index a8fc425..22d5b44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-composite-index-search.aql
@@ -21,14 +21,14 @@ create dataverse index_search;
 use dataverse index_search;
 
 create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
   o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
+  o_clerk: string,
+  o_shippriority: int32,
   o_comment: string
 }
 
@@ -36,12 +36,12 @@ create dataset Orders(OrderType) primary key o_orderkey;
 
 create index idx_Custkey_Orderstatus on Orders(o_custkey, o_orderstatus);
 
-write output to asterix_nc1:"/tmp/index_search.adm";
+set noindexonly 'true';
 
 for $o in dataset('Orders')
 where
   $o.o_custkey = 40 and $o.o_orderstatus = "P"
-return {  
+return {
   "o_orderkey": $o.o_orderkey,
   "o_custkey": $o.o_custkey,
   "o_orderstatus": $o.o_orderstatus

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-index-only.aql
new file mode 100644
index 0000000..26f9d73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-index-only.aql
@@ -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.
+ */
+drop dataverse index_search if exists;
+
+create dataverse index_search;
+
+use dataverse index_search;
+
+create type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+
+create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
+
+create dataset Orders(OrderType)
+  primary key o_orderkey on group1;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
+
+write output to asterix_nc1:"/tmp/index_search.adm";
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40
+return {
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open-index-only.aql
new file mode 100644
index 0000000..2e9095a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open-index-only.aql
@@ -0,0 +1,51 @@
+/*
+ * 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 index_search if exists;
+
+create dataverse index_search;
+
+use dataverse index_search;
+
+create type OrderType as open {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+
+create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
+
+create dataset Orders(OrderType)
+  primary key o_orderkey on group1;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40
+return {
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open.aql
index 90764c3..729b7e4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-open.aql
@@ -23,14 +23,14 @@ create dataverse index_search;
 use dataverse index_search;
 
 create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
   o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
+  o_clerk: string,
+  o_shippriority: int32,
   o_comment: string
 }
 
@@ -42,12 +42,12 @@ create dataset Orders(OrderType)
 
 create index idx_Orders_Custkey on Orders(o_custkey);
 
-write output to asterix_nc1:"/tmp/index_search.adm";
+set noindexonly 'true';
 
 for $o in dataset('Orders')
 where
   $o.o_custkey = 40
-return {  
+return {
   "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey 
+  "o_custkey": $o.o_custkey
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search.aql
index 311eadc..ed8af00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search.aql
@@ -23,14 +23,14 @@ create dataverse index_search;
 use dataverse index_search;
 
 create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
   o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
+  o_clerk: string,
+  o_shippriority: int32,
   o_comment: string
 }
 
@@ -42,12 +42,12 @@ create dataset Orders(OrderType)
 
 create index idx_Orders_Custkey on Orders(o_custkey);
 
-write output to asterix_nc1:"/tmp/index_search.adm";
+set noindexonly 'true';
 
 for $o in dataset('Orders')
 where
   $o.o_custkey = 40
-return {  
+return {
   "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey 
+  "o_custkey": $o.o_custkey
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730-index-only.aql
new file mode 100644
index 0000000..1bf2b2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730-index-only.aql
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use an available rtree index in index subtree.
+ * Issue        : 730
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as {
+screen_name: string,
+lang: string,
+friends_count: int32,
+statuses_count: int32,
+name: string,
+followers_count: int32
+}
+
+create type TweetMessageType as {
+tweetid: int64,
+user: TwitterUserType,
+sender_location: point?,
+send_time: datetime,
+referred_topics: {{ string }},
+message_text: string
+}
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender_location) type rtree;
+
+for $t1 in dataset('TweetMessages')
+where $t1.send_time >= datetime('2011-06-18T14:10:17') and $t1.send_time < datetime('2011-06-18T15:10:17')
+let $n :=  create-circle($t1.sender_location, 5.0)
+return {
+"message": $t1.tweetid,
+"nearby-message": for $t2 in dataset('TweetMessages')
+                             where spatial-intersect($t2.sender_location, $n)
+                             return $t2.tweetid
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730.aql
index 21795b1..c3f9cce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/issue730.aql
@@ -18,7 +18,7 @@
  */
 /*
  * Description  : Test that left-outer-join may use an available rtree index in index subtree.
- * Issue        : 730              
+ * Issue        : 730
  * Expected Res : Success
  * Date         : 8th May 2014
  */
@@ -37,7 +37,7 @@ followers_count: int32
 }
 
 create type TweetMessageType as {
-tweetid: int64,     
+tweetid: int64,
 user: TwitterUserType,
 sender_location: point?,
 send_time: datetime,
@@ -49,7 +49,7 @@ create dataset TweetMessages(TweetMessageType) primary key tweetid;
 
 create index twmSndLocIx on TweetMessages(sender_location) type rtree;
 
-write output to asterix_nc1:"rttest/rtree-index-join_issue730.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 where $t1.send_time >= datetime('2011-06-18T14:10:17') and $t1.send_time < datetime('2011-06-18T15:10:17')

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
new file mode 100644
index 0000000..016edcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01-index-only.aql
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree
+ *              : and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int32,
+    statuses-count: int32,
+    name: string,
+    followers-count: int32
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int32,
+    countB: int32
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+
+for $t1 in dataset('TweetMessages')
+let $n :=  create-circle($t1.sender-location, 0.5)
+where $t1.tweetid < int64("10")
+order by $t1.tweetid
+return {
+"tweetid1": $t1.tweetid,
+"loc1":$t1.sender-location,
+"nearby-message": for $t2 in dataset('TweetMessages')
+                             where spatial-intersect($t2.sender-location, $n)
+                             order by $t2.tweetid
+                             return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location}
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
index 62ca2bb..cf19dce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.aql
@@ -17,7 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree
+ *              : and another for secondary rtree index in index subtree.
  * Issue        : 730, 741
  * Expected Res : Success
  * Date         : 8th May 2014
@@ -55,7 +56,7 @@ create index msgCountAIx on TweetMessages(countA) type btree;
 create index msgCountBIx on TweetMessages(countB) type btree;
 create index msgTextIx on TweetMessages(message-text) type keyword;
 
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 let $n :=  create-circle($t1.sender-location, 0.5)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_04.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_04.aql
new file mode 100644
index 0000000..7d45139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_04.aql
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Self-joins a dataset on the intersection of its point and rectangle attribute.
+ *                  The dataset has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ *                : In fact, this is an index-only plan and the optimized plan should show two paths.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle
+}
+
+create dataset MyData(MyRecord) primary key id;
+
+create index rtree_rec_index on MyData(rec) type rtree;
+
+for $a in dataset('MyData')
+for $b in dataset('MyData')
+where spatial-intersect($a.point, $b.rec)
+return {"a": $a.id, "b": $b.rec}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_05.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_05.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_05.aql
new file mode 100644
index 0000000..6fde6ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_05.aql
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Self-joins a dataset on the intersection of its point attributes.
+ *                  The dataset has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ *                : In fact, this is an index-only plan and the optimized plan should show two paths.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle
+}
+
+create dataset MyData(MyRecord) primary key id;
+
+create index rtree_point_index on MyData(point) type rtree;
+
+for $a in dataset('MyData')
+for $b in dataset('MyData')
+where spatial-intersect($a.point, $b.point)
+return {"a": $a.id, "b": $b.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point-index-only.aql
new file mode 100644
index 0000000..f47d4bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point-index-only.aql
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Self-joins on a dataset on the intersection of a point attribute.
+ *                  The dataset 'TweetMessages' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;
+
+for $t1 in dataset('TweetMessages')
+for $t2 in dataset('TweetMessages')
+let $n :=  create-circle($t1.sender-location, 0.5)
+where spatial-intersect($t2.sender-location, $n)
+order by $t2.tweetid
+return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point.aql
index 816bfe6..9a5eefa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-self-intersect-point.aql
@@ -55,7 +55,7 @@ create index msgCountAIx on TweetMessages(countA) type btree;
 create index msgCountBIx on TweetMessages(countB) type btree;
 create index msgTextIx on TweetMessages(message-text) type keyword;
 
-write output to asterix_nc1:"rttest/index-join_rtree-spatial-self-intersect-point.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 for $t2 in dataset('TweetMessages')

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
new file mode 100644
index 0000000..dae4f0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *                  : However, we set the "noindexonly" option to true. So, the index-only plan should not be triggered.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set noindexonly 'true';
+
+select count(*) from (
+select x.location
+from Fragile_raw x
+where spatial_intersect(x.location, create_polygon([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01.sqlpp
new file mode 100644
index 0000000..7a37b2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-01.sqlpp
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+select count(*) from (
+select x.location
+from Fragile_raw x
+where spatial_intersect(x.location, create_polygon([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
+) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-02.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-02.aql
new file mode 100644
index 0000000..e82a527
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-02.aql
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-03.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-03.aql
new file mode 100644
index 0000000..abb15e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-03.aql
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"point":$o.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-04.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-04.aql
new file mode 100644
index 0000000..27e9509
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-04.aql
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id, "point":$o.point}
+);
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-05.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-05.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-05.aql
new file mode 100644
index 0000000..ad94c18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index/rtree-secondary-index-indexonly-plan-05.aql
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary RTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary RTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+/* For raw Fragile data */
+create type FragileTypeRaw as closed {
+  row_id: int32,
+  sid: int32,
+  date: string,
+  day: int32,
+  time: string,
+  bpm: int32,
+  RR: float,
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+
+};
+
+/* For cleaned Fragile data */
+create type FragileType as closed {
+  row_id: int32,
+  sid: int32,
+  date: date,
+  day: int32,
+  time: time,
+  bpm: int32,
+  RR: float,
+
+  /* new string field and location field*/
+  text: string,
+  location: point,
+  text2: string
+};
+
+/* Create dataset for loading raw Fragile data */
+create dataset Fragile_raw (FragileTypeRaw)
+primary key row_id;
+
+/* Create dataset for cleaned Fragile data */
+create dataset Fragile (FragileType)
+primary key row_id;
+
+
+/* Create rtree secondary index on dataset clean Fragile */
+create index cfLocation on Fragile(location) type rtree;
+create index cfLocation on Fragile_raw(location) type rtree;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+count(
+let $area:=create-rectangle(point("0.0,0.0"), point("4.0,4.0"))
+for $o in dataset('MyData')
+where spatial-intersect($o.point, $area)
+order by $o.id
+return {"id":$o.id}
+);
+


[08/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.aql
new file mode 100644
index 0000000..9555bb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.1.ddl.aql
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.aql
new file mode 100644
index 0000000..56687c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.aql
new file mode 100644
index 0000000..bc5de3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.3.ddl.aql
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+create index rtree_index on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.aql
new file mode 100644
index 0000000..833111c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_02/rtree-spatial-intersect-point_02.4.query.aql
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The inner dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('MyData1')
+for $b in dataset('MyData2')
+where spatial-intersect($a.point, $b.point)
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "apoint": $a.point, "bpoint": $b.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.aql
new file mode 100644
index 0000000..33e7c9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ /*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.aql
new file mode 100644
index 0000000..56687c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.aql
new file mode 100644
index 0000000..ee1eb35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.3.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+create index rtree_index1 on MyData1(point) type rtree;
+create index rtree_index2 on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.aql
new file mode 100644
index 0000000..a5bb5c0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_03/rtree-spatial-intersect-point_03.4.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $a in dataset('MyData1')
+for $b in dataset('MyData2')
+where spatial-intersect($a.point, $b.point)
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "apoint": $a.point, "bpoint": $b.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.aql
new file mode 100644
index 0000000..b3e62ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.aql
new file mode 100644
index 0000000..56687c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.aql
new file mode 100644
index 0000000..249601a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.3.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+create index rtree_index1 on MyData1(poly1) type rtree;
+create index rtree_index2 on MyData2(point) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.aql
new file mode 100644
index 0000000..4d00d76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_04/rtree-spatial-intersect-point_04.4.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $a in dataset('MyData1')
+for $b in dataset('MyData2')
+where spatial-intersect($a.poly1, $b.point)
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "apoly1": $a.poly1, "bpoint": $b.point}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.aql
new file mode 100644
index 0000000..b3e62ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  Both inner and outer dataset have an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join
+ *                  by using an RTree index from the inner dataset.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData1(MyRecord) primary key id;
+create dataset MyData2(MyRecord) primary key id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.aql
new file mode 100644
index 0000000..56687c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.aql
new file mode 100644
index 0000000..bcaa0bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+create index rtree_index1 on MyData1(poly1) type rtree;
+create index rtree_index2 on MyData2(poly2) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.aql
new file mode 100644
index 0000000..ac8a03d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.aql
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $a in dataset('MyData1')
+for $b in dataset('MyData2')
+where spatial-intersect($a.poly1, $b.poly2)
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "apoly1": $a.poly1, "bpoly2": $b.poly2}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.ddl.aql
new file mode 100644
index 0000000..0dfc827
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.1.ddl.aql
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for primary index in index subtree.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.2.update.aql
new file mode 100644
index 0000000..fd0458a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.2.update.aql
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for primary index in index subtree.
+ * Expected Res : Success
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.3.query.aql
new file mode 100644
index 0000000..0512daa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-pidx1/probe-pidx-with-join-btree-pidx1.3.query.aql
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for primary index in index subtree.
+ * Expected Res : Success
+ */
+
+
+use dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.tweetid < int64("10")
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.tweetid /* +indexnl */= $t2.tweetid
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.2.update.aql
index 36deb61..dc4fcdc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.2.update.aql
@@ -18,13 +18,13 @@
  */
 /*
  * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741                 
+ * Issue        : 730, 741
  * Expected Res : Success
  * Date         : 8th May 2014
  */
 
 use dataverse test;
 
-load dataset TweetMessages 
+load dataset TweetMessages
 using localfs
 (("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.3.query.aql
index c8076df..d27955d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx2/probe-pidx-with-join-btree-sidx2.3.query.aql
@@ -18,7 +18,7 @@
  */
 /*
  * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741                 
+ * Issue        : 730, 741
  * Expected Res : Success
  * Date         : 8th May 2014
  */
@@ -31,10 +31,10 @@ order by $t1.tweetid
 return {
 "tweetid1": $t1.tweetid,
 "count1":$t1.countA,
-"t2info": for $t2 in dataset('TweetMessages') 
+"t2info": for $t2 in dataset('TweetMessages')
                         where $t1.countA /* +indexnl */= $t2.countB and
                         $t1.tweetid != $t2.tweetid
-                        order by $t2.tweetid 
+                        order by $t2.tweetid
                         return {"tweetid2": $t2.tweetid,
                                        "count2":$t2.countB}
 };

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.aql
new file mode 100644
index 0000000..a8a8ce6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.aql
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for secondary index in index subtree. For inner branch, this is an index-only plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.aql
new file mode 100644
index 0000000..a045bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.aql
new file mode 100644
index 0000000..54ff9c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.aql
@@ -0,0 +1,32 @@
+/*
+ * 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 dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.tweetid < 10
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.countA /* +indexnl */= $t2.countB
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+});

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
new file mode 100644
index 0000000..8c56ff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available primary index in the index subtree.
+ *                In the probe side, this is an index-only plan since a secondary index-search
+ *                will fetch all fields that are required for the plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    sender-location: point,
+    send-time: datetime,
+    referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.aql
new file mode 100644
index 0000000..a045bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.aql
new file mode 100644
index 0000000..87d68f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.aql
@@ -0,0 +1,32 @@
+/*
+ * 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 dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.countA > 0
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.countA /* +indexnl */= $t2.tweetid
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+});

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
new file mode 100644
index 0000000..c6f8bb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available secondary index in the index subtree.
+ *                In the probe side, this is an index-only plan since a secondary index-search will
+ *                fetch all fields that are required for the plan.
+ *                In the inner branch, this is also an index-only plan since a secondary index-search will
+ *                fetch all fields that are required for the branch.
+ *                All other variables will be propagated from the outer branch.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    sender-location: point,
+    send-time: datetime,
+    referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
new file mode 100644
index 0000000..a045bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
new file mode 100644
index 0000000..0d3f099
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
@@ -0,0 +1,32 @@
+/*
+ * 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 dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.countA > 0
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.countA /* +indexnl */= $t2.countB
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+});

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
new file mode 100644
index 0000000..6241240
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.aql
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available primary index in the index subtree.
+ *                In the probe side, this is a non index-only plan since a secondary index-search can't cover
+ *                all fields that are required for the plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    sender-location: point,
+    send-time: datetime,
+    referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.aql
new file mode 100644
index 0000000..a045bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.aql
new file mode 100644
index 0000000..d1fc6d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.aql
@@ -0,0 +1,32 @@
+/*
+ * 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 dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.countA > 0 and $t1.countB < 10000
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.countA /* +indexnl */= $t2.tweetid
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+});

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
new file mode 100644
index 0000000..dc393a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.aql
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available secondary index in the index subtree.
+ *                In the probe side, this is a non index-only plan since a secondary index-search can't cover
+ *                all fields that are required for the plan.
+ *                In the inner branch, this is a index-only plan since a secondary index-search can cover
+ *                all fields that are required for that branch.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    sender-location: point,
+    send-time: datetime,
+    referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
new file mode 100644
index 0000000..a045bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
new file mode 100644
index 0000000..28e2423
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.aql
@@ -0,0 +1,32 @@
+/*
+ * 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 dataverse test;
+
+count(
+for $t1 in dataset('TweetMessages')
+where $t1.countA > 0 and $t1.countB < 10000
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+                        where $t1.countA /* +indexnl */= $t2.countB
+                        return {"tweetid2": $t2.tweetid,
+                                "count2": $t2.countB}
+});

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.aql
new file mode 100644
index 0000000..11a3605
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.aql
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+//create index btree_index_docid on MyData(docid) type btree;
+//create index btree_index_val1 on MyData(val1) type btree;
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.aql
new file mode 100644
index 0000000..9a7b2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.aql
new file mode 100644
index 0000000..e9c6f51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+order by $o.id
+return {"pk":$o.id, "sk":$o.val1}


[13/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 1c7330a..6f99330 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -19,8 +19,10 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -53,6 +55,7 @@ import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescripto
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -70,7 +73,6 @@ import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -107,33 +109,34 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         DISJUNCTIVE
     }
 
-    private static List<FunctionIdentifier> funcIdents = new ArrayList<>();
-
-    static {
-        funcIdents.add(BuiltinFunctions.STRING_CONTAINS);
-        // For matching similarity-check functions. For example, similarity-jaccard-check returns a list of two items,
-        // and the select condition will get the first list-item and check whether it evaluates to true.
-        funcIdents.add(BuiltinFunctions.GET_ITEM);
-        // Full-text search function
-        funcIdents.add(BuiltinFunctions.FULLTEXT_CONTAINS);
-        funcIdents.add(BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION);
-    }
+    // The second boolean value tells whether the given function generates false positive results.
+    // That is, this function can produce false positive results if it is set to true.
+    // In this case, an index-search alone cannot replace the given SELECT condition and
+    // that SELECT condition needs to be applied after the index-search to get the correct results.
+    // Currently, only full-text index search does not generate false positive results.
+    private static final List<Pair<FunctionIdentifier, Boolean>> FUNC_IDENTIFIERS = Collections.unmodifiableList(
+            Arrays.asList(new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.STRING_CONTAINS, true),
+                    // For matching similarity-check functions. For example, similarity-jaccard-check returns
+                    // a list of two items, and the select condition will get the first list-item and
+                    // check whether it evaluates to true.
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.GET_ITEM, true),
+                    // Full-text search function
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.FULLTEXT_CONTAINS, false),
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION, false)));
 
     // These function identifiers are matched in this AM's analyzeFuncExprArgs(),
     // and are not visible to the outside driver.
-    private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<>();
-
-    static {
-        secondLevelFuncIdents.add(BuiltinFunctions.SIMILARITY_JACCARD_CHECK);
-        secondLevelFuncIdents.add(BuiltinFunctions.EDIT_DISTANCE_CHECK);
-        secondLevelFuncIdents.add(BuiltinFunctions.EDIT_DISTANCE_CONTAINS);
-    }
+    private static final List<Pair<FunctionIdentifier, Boolean>> SECOND_LEVEL_FUNC_IDENTIFIERS =
+            Collections.unmodifiableList(Arrays.asList(
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.SIMILARITY_JACCARD_CHECK, true),
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.EDIT_DISTANCE_CHECK, true),
+                    new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.EDIT_DISTANCE_CONTAINS, true)));
 
     public static InvertedIndexAccessMethod INSTANCE = new InvertedIndexAccessMethod();
 
     @Override
-    public List<FunctionIdentifier> getOptimizableFunctions() {
-        return funcIdents;
+    public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions() {
+        return FUNC_IDENTIFIERS;
     }
 
     @Override
@@ -224,10 +227,21 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                 }
             }
         }
-        // Check that the matched function is optimizable by this access method.
-        if (!secondLevelFuncIdents.contains(matchedFuncExpr.getFunctionIdentifier())) {
+        // Checks that the matched function is optimizable by this access method.
+        boolean found = false;
+        for (Iterator<Pair<FunctionIdentifier, Boolean>> iterator = SECOND_LEVEL_FUNC_IDENTIFIERS.iterator(); iterator
+                .hasNext();) {
+            FunctionIdentifier fID = iterator.next().first;
+
+            if (fID != null && matchedFuncExpr != null && fID.equals(matchedFuncExpr.getFunctionIdentifier())) {
+                found = true;
+                break;
+            }
+        }
+        if (!found) {
             return false;
         }
+
         boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assignsAndUnnests,
                 matchedAssignOrUnnestIndex, analysisCtx);
         boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assignsAndUnnests,
@@ -376,10 +390,18 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
     }
 
     @Override
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
-            IOptimizationContext context) throws AlgebricksException {
+    public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTopOpRefs, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+        // TODO: we currently do not support the index-only plan for the inverted index searches since
+        // there can be many <SK, PK> pairs for the same PK and we may see two different records with the same PK
+        // (e.g., the record is deleted and inserted with the same PK). The reason is that there are
+        // no locking processes during a secondary index DML operation. When a secondary index search can see
+        // the only one version of the record during the lifetime of a query, index-only plan can be applied.
+        boolean generateInstantTrylockResultFromIndexSearch = false;
 
         IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
         Dataset dataset = indexSubTree.getDataset();
@@ -412,6 +434,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
             inputOp.getInputs().add(new MutableObject<>(
                     OperatorManipulationUtil.deepCopy(dataSourceScan.getInputs().get(0).getValue())));
+            context.computeAndSetTypeEnvironmentForOperator(inputOp);
             inputOp.setExecutionMode(dataSourceScan.getExecutionMode());
         } else {
             // We are optimizing a join. Add the input variable to the secondaryIndexFuncArgs.
@@ -420,13 +443,17 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             inputOp = (AbstractLogicalOperator) probeSubTree.getRoot();
         }
         jobGenParams.setKeyVarList(keyVarList);
+        // By default, we don't generate SK output for an inverted index
+        // since it doesn't contain a field value, only part of it.
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
-                metaRecordType, chosenIndex, inputOp, jobGenParams, context, true, retainInput, retainNull);
+                metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainNull,
+                generateInstantTrylockResultFromIndexSearch);
 
-        // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        AbstractUnnestMapOperator primaryIndexUnnestOp =
-                AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType, metaRecordType,
-                        secondaryIndexUnnestOp, context, true, retainInput, retainNull, false);
+        // Generates the rest of the upstream plan which feeds the search results into the primary index.
+        ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
+                conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
+                secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
+                indexSubTree, newNullPlaceHolderForLOJ);
 
         return primaryIndexUnnestOp;
     }
@@ -451,23 +478,24 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
     public boolean applySelectPlanTransformation(List<Mutable<ILogicalOperator>> afterSelectRefs,
             Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree, Index chosenIndex,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(null, subTree, null, chosenIndex, analysisCtx,
-                AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
-                        afterSelectRefs),
-                false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
-                        .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                context);
+        SelectOperator selectOp = (SelectOperator) selectRef.getValue();
+        ILogicalOperator indexPlanRootOp =
+                createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
+                        subTree.getAssignsAndUnnestsRefs(),
+                        subTree, null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue()
+                                .getInputs().get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
+                        context, null);
+
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.getDataSourceRef().setValue(indexPlanRootOp);
         return true;
     }
 
     @Override
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
-            boolean hasGroupBy) throws AlgebricksException {
-        // Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
+    public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
+            OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean hasGroupBy) throws AlgebricksException {
         Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
         OptimizableOperatorSubTree indexSubTree;
         OptimizableOperatorSubTree probeSubTree;
@@ -485,7 +513,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
 
         IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
         // The arguments of edit-distance-contains() function are asymmetrical, we can only use index
-        // if the dataset of index subtree and the dataset of first argument's subtree is the same
+        // if the dataset of index subtree and the dataset of first argument's subtree is the same.
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CONTAINS
                 && optFuncExpr.getOperatorSubTree(0).getDataset() != null && !optFuncExpr.getOperatorSubTree(0)
                         .getDataset().getDatasetName().equals(indexSubTree.getDataset().getDatasetName())) {
@@ -500,7 +528,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
 
             //reset the null place holder variable
-            AccessMethodUtils.resetLOJNullPlaceholderVariableInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+            AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
         }
 
         AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) joinRef.getValue();
@@ -535,8 +563,9 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             probeSubTree.setRoot(newProbeRootRef.getValue());
         }
         // Create regular indexed-nested loop join path.
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(null, indexSubTree, probeSubTree, chosenIndex,
-                analysisCtx, true, isLeftOuterJoin, true, context);
+        ILogicalOperator indexPlanRootOp = createIndexSearchPlan(afterJoinRefs, joinRef,
+                new MutableObject<ILogicalExpression>(joinCond), indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree,
+                probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context, newNullPlaceHolderVar);
         indexSubTree.getDataSourceRef().setValue(indexPlanRootOp);
 
         // Change join into a select with the same condition.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 2534680..7c2edb9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -19,7 +19,9 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -61,6 +63,7 @@ public class OptimizableOperatorSubTree {
         EXTERNAL_SCAN,
         PRIMARY_INDEX_LOOKUP,
         COLLECTION_SCAN,
+        INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
         NO_DATASOURCE
     }
 
@@ -75,6 +78,9 @@ public class OptimizableOperatorSubTree {
     private Dataset dataset = null;
     private ARecordType recordType = null;
     private ARecordType metaRecordType = null;
+    // Contains the field names for all assign operations in this sub-tree.
+    // This will be used for the index-only plan check.
+    private Map<LogicalVariable, List<String>> varsToFieldNameMap = new HashMap<>();
 
     // Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
     // (E.g. There are index-nested-loop-joins in the plan.)
@@ -83,6 +89,9 @@ public class OptimizableOperatorSubTree {
     private List<Dataset> ixJoinOuterAdditionalDatasets = null;
     private List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
 
+    /**
+     * Identifies the root of the subtree and initializes the data-source, assign, and unnest information.
+     */
     public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) throws AlgebricksException {
         reset();
         rootRef = subTreeOpRef;
@@ -160,15 +169,11 @@ public class OptimizableOperatorSubTree {
                             AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                             jobGenParams.readFromFuncArgs(f.getArguments());
                             if (jobGenParams.isPrimaryIndex()) {
-                                if (getDataSourceRef() == null) {
-                                    setDataSourceRef(subTreeOpRef);
-                                    setDataSourceType(DataSourceType.PRIMARY_INDEX_LOOKUP);
-                                } else {
-                                    // One datasource already exists. This is an additional datasource.
-                                    initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
-                                    getIxJoinOuterAdditionalDataSourceTypes().add(DataSourceType.PRIMARY_INDEX_LOOKUP);
-                                    getIxJoinOuterAdditionalDataSourceRefs().add(subTreeOpRef);
-                                }
+                                intializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
+                                dataSourceFound = true;
+                            } else if (unnestMapOp.getGenerateCallBackProceedResultVar()) {
+                                intializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
+                                        subTreeOpRef);
                                 dataSourceFound = true;
                             }
                         } else if (f.getFunctionIdentifier().equals(BuiltinFunctions.EXTERNAL_LOOKUP)) {
@@ -213,6 +218,18 @@ public class OptimizableOperatorSubTree {
         return false;
     }
 
+    private void intializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
+        if (getDataSourceRef() == null) {
+            setDataSourceRef(opRef);
+            setDataSourceType(dsType);
+        } else {
+            // One datasource already exists. This is an additional datasource.
+            initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
+            getIxJoinOuterAdditionalDataSourceTypes().add(dsType);
+            getIxJoinOuterAdditionalDataSourceRefs().add(opRef);
+        }
+    }
+
     /**
      * Find the dataset corresponding to the datasource scan in the metadata.
      * Also sets recordType to be the type of that dataset.
@@ -254,6 +271,7 @@ public class OptimizableOperatorSubTree {
                     datasetName = datasetInfo.second;
                     break;
                 case PRIMARY_INDEX_LOOKUP:
+                case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
                     AbstractUnnestOperator unnestMapOp = (AbstractUnnestOperator) sourceOpRefs.get(i).getValue();
                     ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
@@ -369,7 +387,7 @@ public class OptimizableOperatorSubTree {
     }
 
     /**
-     * Get primary key variables from the given data-source.
+     * Gets the primary key variables from the given data-source.
      */
     public void getPrimaryKeyVars(Mutable<ILogicalOperator> dataSourceRefToFetch, List<LogicalVariable> target)
             throws AlgebricksException {
@@ -389,12 +407,26 @@ public class OptimizableOperatorSubTree {
                 primaryKeys = AccessMethodUtils.getPrimaryKeyVarsFromPrimaryUnnestMap(dataset, unnestMapOp);
                 target.addAll(primaryKeys);
                 break;
+            case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
+                AbstractUnnestMapOperator idxOnlyPlanUnnestMapOp =
+                        (AbstractUnnestMapOperator) dataSourceRefToFetchKey.getValue();
+                List<LogicalVariable> idxOnlyPlanKeyVars = idxOnlyPlanUnnestMapOp.getVariables();
+                int indexOnlyPlanNumPrimaryKeys = dataset.getPrimaryKeys().size();
+                // The order of variables: SK, PK, the result of instantTryLock on PK.
+                // The last variable keeps the result of instantTryLock on PK.
+                // Thus, we deduct 1 to only count key variables.
+                int start = idxOnlyPlanKeyVars.size() - 1 - indexOnlyPlanNumPrimaryKeys;
+                int end = start + indexOnlyPlanNumPrimaryKeys;
+
+                for (int i = start; i < end; i++) {
+                    target.add(idxOnlyPlanKeyVars.get(i));
+                }
+                break;
             case EXTERNAL_SCAN:
                 break;
             case NO_DATASOURCE:
             default:
                 throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE);
-
         }
     }
 
@@ -405,6 +437,11 @@ public class OptimizableOperatorSubTree {
             case PRIMARY_INDEX_LOOKUP:
                 AbstractScanOperator scanOp = (AbstractScanOperator) getDataSourceRef().getValue();
                 return scanOp.getVariables();
+            case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
+                // This data-source doesn't have record variables.
+                List<LogicalVariable> pkVars = new ArrayList<>();
+                getPrimaryKeyVars(dataSourceRef, pkVars);
+                return pkVars;
             case COLLECTION_SCAN:
                 return new ArrayList<>();
             case NO_DATASOURCE:
@@ -422,6 +459,10 @@ public class OptimizableOperatorSubTree {
                     AbstractScanOperator scanOp =
                             (AbstractScanOperator) getIxJoinOuterAdditionalDataSourceRefs().get(idx).getValue();
                     return scanOp.getVariables();
+                case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
+                    List<LogicalVariable> PKVars = new ArrayList<>();
+                    getPrimaryKeyVars(ixJoinOuterAdditionalDataSourceRefs.get(idx), PKVars);
+                    return PKVars;
                 case COLLECTION_SCAN:
                     return new ArrayList<>();
                 case NO_DATASOURCE:
@@ -539,4 +580,8 @@ public class OptimizableOperatorSubTree {
         this.ixJoinOuterAdditionalRecordTypes = ixJoinOuterAdditionalRecordTypes;
     }
 
+    public Map<LogicalVariable, List<String>> getVarsToFieldNameMap() {
+        return varsToFieldNameMap;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index fd46194..f431603 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
@@ -31,15 +33,19 @@ import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Quadruple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
@@ -49,27 +55,31 @@ import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 /**
  * Class for helping rewrite rules to choose and apply RTree indexes.
  */
 public class RTreeAccessMethod implements IAccessMethod {
 
-    private static List<FunctionIdentifier> funcIdents = new ArrayList<>();
-
-    static {
-        funcIdents.add(BuiltinFunctions.SPATIAL_INTERSECT);
-    }
+    // The second boolean value tells whether the given function generates false positive results.
+    // That is, this function can produce false positive results if it is set to true.
+    // In this case, an index-search alone cannot replace the given SELECT condition and
+    // that SELECT condition needs to be applied after the index-search to get the final results.
+    // In R-Tree case, depending on the parameters of the SPATIAL_INTERSECT function, it may/may not produce
+    // false positive results. Thus, we need to have one more step to check whether the SPATIAL_INTERSECT generates
+    // false positive results or not.
+    private static final List<Pair<FunctionIdentifier, Boolean>> FUNC_IDENTIFIERS = Collections.unmodifiableList(
+            Arrays.asList(new Pair<FunctionIdentifier, Boolean>(BuiltinFunctions.SPATIAL_INTERSECT, true)));
 
     public static final RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
 
     @Override
-    public List<FunctionIdentifier> getOptimizableFunctions() {
-        return funcIdents;
+    public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions() {
+        return FUNC_IDENTIFIERS;
     }
 
     @Override
@@ -98,87 +108,110 @@ public class RTreeAccessMethod implements IAccessMethod {
     public boolean applySelectPlanTransformation(List<Mutable<ILogicalOperator>> afterSelectRefs,
             Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree, Index chosenIndex,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
+        SelectOperator selectOp = (SelectOperator) selectRef.getValue();
+        Mutable<ILogicalExpression> conditionRef = selectOp.getCondition();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) conditionRef.getValue();
+        ARecordType recordType = subTree.getRecordType();
+
         // TODO: We can probably do something smarter here based on selectivity or MBR area.
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, analysisCtx,
-                AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
-                        afterSelectRefs),
-                false, false, context);
-        if (primaryIndexUnnestOp == null) {
+        IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
+
+        int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
+                optFuncExpr.getFieldName(optFieldIdx), recordType);
+        if (keyPairType == null) {
             return false;
         }
-        // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
-        subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-        return true;
-    }
 
-    @Override
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
-            IOptimizationContext context) throws AlgebricksException {
-        return createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex, analysisCtx, retainInput,
-                retainNull, requiresBroadcast, context);
-    }
+        // To check whether the given plan is an index-only plan:
+        // index-only plan possible?
+        boolean isIndexOnlyPlan = false;
 
-    @Override
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
-            boolean hasGroupBy) throws AlgebricksException {
-        // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left
-        // side).
-        Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
-        OptimizableOperatorSubTree indexSubTree;
-        OptimizableOperatorSubTree probeSubTree;
+        // secondary key field usage after the select operator
+        boolean secondaryKeyFieldUsedAfterSelectOp = false;
 
-        // We assume that the left subtree is the outer branch and the right subtree is the inner branch.
-        // This assumption holds true since we only use an index from the right subtree.
-        // The following is just a sanity check.
-        if (rightSubTree.hasDataSourceScan()
-                && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
-            indexSubTree = rightSubTree;
-            probeSubTree = leftSubTree;
-        } else {
+        // Whether a verification is required after the secondary index search
+        // In other words, can the chosen method generate any false positive results?
+        boolean requireVerificationAfterSIdxSearch = false;
+        Pair<Boolean, Boolean> functionFalsePositiveCheck =
+                AccessMethodUtils.canFunctionGenerateFalsePositiveResultsUsingIndex(funcExpr, FUNC_IDENTIFIERS);
+
+        if (!functionFalsePositiveCheck.first) {
             return false;
         }
 
-        LogicalVariable newNullPlaceHolderVar = null;
-        if (isLeftOuterJoin) {
-            // get a new null place holder variable that is the first field variable of the primary key
-            // from the indexSubTree's datasourceScanOp
-            newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+        // Does the given index can cover all search predicates?
+        boolean doesSIdxSearchCoverAllPredicates = false;
+
+        // Preliminary check for the index-only plan for R-Tree:
+        // If the given index is not built on a POINT or a RECTANGLE field,
+        // the query result can include false positives. And the result from secondary index search is an MBR,
+        // thus we can't construct original secondary field value to remove any false positive results.
+        if (keyPairType.first.getTypeTag() == BuiltinType.APOINT.getTypeTag()
+                || keyPairType.first.getTypeTag() == BuiltinType.ARECTANGLE.getTypeTag()) {
+            isIndexOnlyPlan = true;
+            // The following variable can be changed if a query shape is not a POINT or rectangle.
+            requireVerificationAfterSIdxSearch = false;
+        } else {
+            isIndexOnlyPlan = false;
+            requireVerificationAfterSIdxSearch = true;
         }
 
-        // TODO: We can probably do something smarter here based on selectivity or MBR area.
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex,
-                analysisCtx, true, isLeftOuterJoin, true, context);
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo =
+                new Quadruple<>(isIndexOnlyPlan, secondaryKeyFieldUsedAfterSelectOp, requireVerificationAfterSIdxSearch,
+                        doesSIdxSearchCoverAllPredicates);
+
+        Dataset dataset = subTree.getDataset();
+
+        // Is this plan an index-only plan?
+        if (isIndexOnlyPlan) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+                AccessMethodUtils.indexOnlyPlanCheck(afterSelectRefs, selectRef, subTree, null, chosenIndex,
+                        analysisCtx, context, indexOnlyPlanInfo);
+                isIndexOnlyPlan = indexOnlyPlanInfo.getFirst();
+            } else {
+                // An index on an external dataset can't be optimized for the index-only plan.
+                isIndexOnlyPlan = false;
+                indexOnlyPlanInfo.setFirst(isIndexOnlyPlan);
+            }
+        }
+
+        analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
+
+        ILogicalOperator primaryIndexUnnestOp = createIndexSearchPlan(afterSelectRefs, selectRef,
+                selectOp.getCondition(), subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
+                AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
+                        afterSelectRefs),
+                false, false, context, null);
+
         if (primaryIndexUnnestOp == null) {
             return false;
         }
 
-        if (isLeftOuterJoin && hasGroupBy) {
-            // reset the null place holder variable
-            AccessMethodUtils.resetLOJNullPlaceholderVariableInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+        // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
+        if (!isIndexOnlyPlan || dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
+        } else {
+            // If this is an index-only plan, the topmost operator returned is UNIONALL operator.
+            if (primaryIndexUnnestOp.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
+                selectRef.setValue(primaryIndexUnnestOp);
+            } else {
+                subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
+            }
         }
-
-        indexSubTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-        // Change join into a select with the same condition.
-        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
-        SelectOperator topSelect = new SelectOperator(joinOp.getCondition(), isLeftOuterJoin, newNullPlaceHolderVar);
-        topSelect.getInputs().add(indexSubTree.getRootRef());
-        topSelect.setExecutionMode(ExecutionMode.LOCAL);
-        context.computeAndSetTypeEnvironmentForOperator(topSelect);
-        // Replace the original join with the new subtree rooted at the select op.
-        joinRef.setValue(topSelect);
         return true;
     }
 
-    private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree,
+    @Override
+    public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopRefs,
+            Mutable<ILogicalOperator> topRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTopRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context)
-            throws AlgebricksException {
-
+            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+        // TODO: We can probably do something smarter here based on selectivity or MBR area.
         IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
+
         Dataset dataset = indexSubTree.getDataset();
         ARecordType recordType = indexSubTree.getRecordType();
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -195,43 +228,48 @@ public class RTreeAccessMethod implements IAccessMethod {
         IAType spatialType = keyPairType.first;
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         int numSecondaryKeys = numDimensions * 2;
-        // we made sure indexSubTree has datasource scan
+
+        Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
+        boolean isIndexOnlyPlan = indexOnlyPlanInfo.getFirst();
+        // We apply index-only plan for an internal dataset.
+        boolean generateInstantTrylockResultFromIndexSearch =
+                dataset.getDatasetType() == DatasetType.INTERNAL && isIndexOnlyPlan ? true : false;
+
+        // We made sure that the indexSubTree has a datasource scan.
         AbstractDataSourceOperator dataSourceOp =
                 (AbstractDataSourceOperator) indexSubTree.getDataSourceRef().getValue();
         RTreeJobGenParams jobGenParams = new RTreeJobGenParams(chosenIndex.getIndexName(), IndexType.RTREE,
                 dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // A spatial object is serialized in the constant of the func expr we are optimizing.
         // The R-Tree expects as input an MBR represented with 1 field per dimension.
-        // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the
-        // R-Tree expects them).
-        // List of variables for the assign.
+        // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple
+        // (as the R-Tree expects them). List of variables for the assign.
         ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
         // List of expressions for the assign.
         ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<>();
-        Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(chosenIndex,
-                optFuncExpr, optFieldType, indexSubTree, probeSubTree);
-        ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
+        ILogicalExpression returnedSearchKeyExpr =
+                AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, optFieldType, probeSubTree).first;
 
         for (int i = 0; i < numSecondaryKeys; i++) {
             // The create MBR function "extracts" one field of an MBR around the given spatial object.
             AbstractFunctionCallExpression createMBR =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
             // Spatial object is the constant from the func expr we are optimizing.
-            createMBR.getArguments().add(new MutableObject<>(searchKeyExpr));
-            // The number of dimensions.
+            createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr));
+            // The number of dimensions
             createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(numDimensions)))));
-            // Which part of the MBR to extract.
+            // Which part of the MBR to extract?
             createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
-            // Add a variable and its expr to the lists which will be passed into an assign op.
+            // Adds a variable and its expr to the lists which will be passed into an assign op.
             LogicalVariable keyVar = context.newVar();
             keyVarList.add(keyVar);
             keyExprList.add(new MutableObject<ILogicalExpression>(createMBR));
         }
         jobGenParams.setKeyVarList(keyVarList);
 
-        // Assign operator that "extracts" the MBR fields from the func-expr constant into a tuple.
+        // Assigns an operator that "extracts" the MBR fields from the func-expr constant into a tuple.
         AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
         if (probeSubTree == null) {
             // We are optimizing a selection query.
@@ -242,17 +280,77 @@ public class RTreeAccessMethod implements IAccessMethod {
         } else {
             // We are optimizing a join, place the assign op top of the probe subtree.
             assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
+            assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
+            OperatorPropertiesUtil.typeOpRec(probeSubTree.getRootRef(), context);
         }
+        context.computeAndSetTypeEnvironmentForOperator(assignSearchKeys);
 
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
-                metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput, retainNull);
+                metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, retainInput, retainNull,
+                generateInstantTrylockResultFromIndexSearch);
 
-        // Generate the rest of the upstream plan which feeds the search results into the primary index.
+        // Generates the rest of the upstream plan which feeds the search results into the primary index.
         return dataset.getDatasetType() == DatasetType.EXTERNAL
-                ? AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp, dataset, recordType,
-                        secondaryIndexUnnestOp, context, retainInput, retainNull)
-                : AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp, dataset, recordType, metaRecordType,
-                        secondaryIndexUnnestOp, context, true, retainInput, false, false);
+                ? AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp, dataset, recordType, metaRecordType,
+                        secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull)
+                : AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
+                        dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
+                        retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree,
+                        newNullPlaceHolderForLOJ);
+    }
+
+    @Override
+    public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+            Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
+            OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean hasGroupBy) throws AlgebricksException {
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
+        Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
+
+        AbstractFunctionCallExpression funcExpr = null;
+        if (conditionRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            funcExpr = (AbstractFunctionCallExpression) conditionRef.getValue();
+        }
+
+        Dataset dataset = analysisCtx.getIndexDatasetMap().get(chosenIndex);
+
+        // Determine if the index is applicable on the right (inner) side.
+        OptimizableOperatorSubTree indexSubTree = null;
+        OptimizableOperatorSubTree probeSubTree = null;
+        // We assume that the left subtree is the outer branch and the right subtree is the inner branch.
+        // This assumption holds true since we only use an index from the right subtree.
+        // The following is just a sanity check.
+        if (rightSubTree.hasDataSourceScan()
+                && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
+            indexSubTree = rightSubTree;
+            probeSubTree = leftSubTree;
+        } else {
+            return false;
+        }
+
+        LogicalVariable newNullPlaceHolderVar = null;
+        if (isLeftOuterJoin) {
+            // Gets a new null place holder variable that is the first field variable of the primary key
+            // from the indexSubTree's datasourceScanOp.
+            newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+        }
+
+        boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
+                chosenIndex, analysisCtx, context, funcExpr, FUNC_IDENTIFIERS);
+        if (!canContinue) {
+            return false;
+        }
+
+        ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
+                indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
+                isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+
+        if (indexSearchOp == null) {
+            return false;
+        }
+
+        return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, analysisCtx,
+                context, isLeftOuterJoin, hasGroupBy, indexSearchOp, newNullPlaceHolderVar, conditionRef, dataset);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/data/nontagged/customerData2.json
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/nontagged/customerData2.json b/asterixdb/asterix-app/data/nontagged/customerData2.json
new file mode 100644
index 0000000..7624a71
--- /dev/null
+++ b/asterixdb/asterix-app/data/nontagged/customerData2.json
@@ -0,0 +1,6 @@
+{  "cid": 775,  "name": "Jodi Rotruck", "cashBack": 775, "age": null, "address": {  "number": 8389,  "street": "Hill St.",  "city": "Mountain View" },  "lastorder": {  "oid": 66,  "total": 38.618626f } }
+{  "cid": 5,  "name": "Jodi Alex",  "cashBack": 5, "age": 19, "address": null, "lastorder": {  "oid": 48,  "total": 318.618626f } }
+{  "cid": 1,  "name": "Mike Carey",  "cashBack": 1, "address": {  "number": 389,  "street": "Hill St.",  "city": "Mountain View" },  "lastorder": {  "oid": 18,  "total": 338.618626f } }
+{  "cid": 0,  "name": "Mike ley",  "cashBack": 0, "address": null, "lastorder": {  "oid": 0258,  "total": 368.618626f } }
+{  "cid": 4,  "name": "Mary Carey",  "cashBack": 4, "age": 12,  "address": {  "number": 8,  "street": "Hill St.",  "city": "Mountain View" },  "lastorder": {  "oid": 4545,  "total": 87.618626f } }
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 189a7e1..c15704a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -64,6 +64,7 @@ import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.am.AbstractIntroduceAccessMethodRule;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
@@ -127,7 +128,7 @@ public class APIFramework {
                     FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
                     FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
                     FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS, "inline_with",
-                    "hash_merge", "output-record-type");
+                    "hash_merge", "output-record-type", AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION);
 
     private final IRewriterFactory rewriterFactory;
     private final IAstPrintVisitorFactory astPrintVisitorFactory;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.aql
new file mode 100644
index 0000000..f2c4028
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01-index-only.aql
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree
+ *              : and another for secondary btree index in index subtree.
+ *              : In fact, this is an index-only plan from the inner branch's perspective since only PK and SK
+ *              : variables are used and returned.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int32,
+    statuses-count: int32,
+    name: string,
+    followers-count: int32
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int32,
+    countB: int32
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;
+
+for $t1 in dataset('TweetMessages')
+where $t1.tweetid < int64("10")
+order by $t1.tweetid
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+          where $t1.countA /* +indexnl */= $t2.countB
+          order by $t2.tweetid
+          return {"tweetid2": $t2.tweetid,
+                  "count2":$t2.countB}
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.aql
index 38032f5..71e94e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.aql
@@ -17,7 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *              : another for secondary btree index in index subtree.
  * Issue        : 730, 741
  * Expected Res : Success
  * Date         : 8th May 2014
@@ -55,7 +56,7 @@ create index msgCountAIx on TweetMessages(countA) type btree;
 create index msgCountBIx on TweetMessages(countB) type btree;
 create index msgTextIx on TweetMessages(message-text) type keyword;
 
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 where $t1.tweetid < int64("10")

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.aql
new file mode 100644
index 0000000..08e786f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-to-secondary-indexonly-plan-equi-join_01.aql
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  In fact, this is an index-only plan.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+
+count(
+for $c in dataset('Customers')
+for $o in dataset('Orders')
+where $c.cid /*+ indexnl */ = $o.cid
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_04.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_04.sqlpp
new file mode 100644
index 0000000..15107b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_04.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins three datasets.
+ *                : Since the given dataset in the inner branch has a secondary index on the field
+ *                : that is being joined, we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create dataset testdst(TestType) primary key id;
+create dataset testdst2(TestType) primary key id;
+create dataset testdst3(TestType) primary key id;
+
+create index sec3_Idx on testdst3(val) type btree;
+
+SELECT * FROM
+testdst a JOIN testdst2 b ON a.val = b.val JOIN testdst3 c ON b.val /* +indexnl */ =  c.val;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_05.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_05.sqlpp
new file mode 100644
index 0000000..31fc953
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_05.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins a constant array with a dataset.
+ *                : Since the given dataset in the inner branch has a secondary index on the field that is being joined,
+ *                : we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (val) type btree;
+
+SELECT * FROM
+[1, 2, 3] AS bar JOIN testdst ON bar /* +indexnl */ = testdst.val;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_06.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_06.sqlpp
new file mode 100644
index 0000000..558e172
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_06.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Joins two datasets.
+ *                : Since the given dataset in the inner branch has a secondary index on the field that is being joined,
+ *                : we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create  dataset testdst(TestType) primary key id;
+create  dataset testdst2(TestType) primary key id;
+
+create  index sec_Idx on testdst2(val) type btree;
+
+SELECT * FROM
+(SELECT val, COUNT(*) FROM testdst GROUP BY val) AS bar JOIN testdst2 ON bar.val /* +indexnl */ = testdst2.val;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.aql
new file mode 100644
index 0000000..d001bc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-primary-equi-join_01.aql
@@ -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.
+ */
+
+ /*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ *                  An index-only plan exists in the outer branch.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 800 and $o.cid /*+ indexnl */ = $c.cid
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
new file mode 100644
index 0000000..44efb8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Each branch (outer and inner) will be transformed as an index-only plan.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 100000 and $o.cid /*+ indexnl */ = $c.cashBack
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
new file mode 100644
index 0000000..950e451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-non-indexonly-plan-to-secondary-indexonly-plan-equi-join_01.aql
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Orders' secondary index.
+ *                  Inner branch will be transformed as an index-only plan.
+ *                  Outer branch cannot be transformed as an index-only plan as an index can't cover
+ *                  all search predicates even excluding the join condition.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int64,
+  name: string,
+  cashBack: int64,
+  age: int64?,
+  address: AddressType?,
+  lastorder: {
+    oid: int64,
+    total: float
+  }
+}
+
+create type OrderType as open {
+  oid: int64,
+  cid: int64,
+  orderstatus: string,
+  orderpriority: string,
+  clerk: string,
+  total: float,
+  items: [int64]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset Orders(OrderType) primary key oid;
+
+create index CustomerID_idx on Orders(cid);
+create index Cashback_idx on Customers(cashBack);
+
+count(
+for $o in dataset('Orders')
+for $c in dataset('Customers')
+where $o.cid < 100000 and $o.total >= 0 and $o.cid /*+ indexnl */ = $c.cashBack
+return {"oid": $o.oid, "cid":$c.cid}
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join-index-only.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join-index-only.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join-index-only.aql
new file mode 100644
index 0000000..e71101a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join-index-only.aql
@@ -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.
+ */
+/*
+ * Description    : Self-equi joins on a dataset using two fields - countA and countB.
+ *                  TweetMessages has a secondary btree index on countB, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  In fact, this is an index-only plan from the inner branch's perspective since only PK and SK
+ *                  variables are used and returned.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends-count: int64,
+    statuses-count: int64,
+    name: string,
+    followers-count: int64
+}
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+        user: TwitterUserType,
+        sender-location: point,
+    send-time: datetime,
+        referred-topics: {{ string }},
+    message-text: string,
+    countA: int64,
+    countB: int64
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(message-text) type keyword;
+
+for $t1 in dataset('TweetMessages')
+for $t2 in dataset('TweetMessages')
+let $c := $t1.countA + 20
+where $c /* +indexnl */= $t2.countB
+order by $t2.tweetid
+return {"tweetid2": $t2.tweetid, "count2":$t2.countB};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join.aql
index abfd197..74f2d87 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-self-equi-join.aql
@@ -55,7 +55,7 @@ create index msgCountAIx on TweetMessages(countA) type btree;
 create index msgCountBIx on TweetMessages(countB) type btree;
 create index msgTextIx on TweetMessages(message-text) type keyword;
 
-write output to asterix_nc1:"rttest/btree-index-join_self-secondary-equi-join.adm";
+set noindexonly 'true';
 
 for $t1 in dataset('TweetMessages')
 for $t2 in dataset('TweetMessages')

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-composite-index-indexonly-plan-01.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-composite-index-indexonly-plan-01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-composite-index-indexonly-plan-01.aql
new file mode 100644
index 0000000..2599bd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-composite-index-indexonly-plan-01.aql
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+for $o in dataset('MyData')
+where $o.docid < 3
+return {"pk":$o.id, "sk":$o.val1}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
new file mode 100644
index 0000000..0be684e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01-disable-indexonly-plan.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *                  : However, we set the "noindexonly" option to true. So, the index-only plan should not be triggered.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set noindexonly 'true';
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01.sqlpp
new file mode 100644
index 0000000..af1a099
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-index-indexonly-plan-01.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;


[05/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.sqlpp
new file mode 100644
index 0000000..9116663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+load dataset MyData1
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+
+load dataset MyData2
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.ddl.sqlpp
new file mode 100644
index 0000000..d283b1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.3.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.
+ */
+
+use test;
+
+create index rtree_index1 on MyData1(poly1) type rtree;
+create index rtree_index2 on MyData2(poly2) type rtree;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.sqlpp
new file mode 100644
index 0000000..3f8f9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/rtree-spatial-intersect-point_05/rtree-spatial-intersect-point_05.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {'aid':a.id,'bid':b.id,'apoly1':a.poly1,'bpoly2':b.poly2}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.poly1,b.poly2)
+order by a.id,b.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.sqlpp
new file mode 100644
index 0000000..4920706
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for secondary index in index subtree. For inner branch, this is an index-only plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TwitterUserType as closed {
+    `screen-name`: string,
+    lang: string,
+    `friends-count`: int64,
+    `statuses-count`: int64,
+    name: string,
+    `followers-count`: int64
+};
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    `sender-location`: point,
+    `send-time`: datetime,
+    `referred-topics`: {{ string }},
+    `message-text`: string,
+    countA: int64,
+    countB: int64
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(`sender-location`) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(`message-text`) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.sqlpp
new file mode 100644
index 0000000..cb2a156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.sqlpp
new file mode 100644
index 0000000..65d953c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-pidx-with-join-btree-sidx3-indexonly-plan/probe-pidx-with-join-btree-sidx3-indexonly-plan.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where t1.countA /*+ indexnl */ = t2.countB
+    )}
+from  TweetMessages as t1
+where t1.tweetid < test.bigint('10')) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
new file mode 100644
index 0000000..adffdfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available primary index in the index subtree.
+ *                In the probe side, this is an index-only plan since a secondary index-search
+ *                will fetch all fields that are required for the plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TwitterUserType as closed {
+    `screen-name`: string,
+    lang: string,
+    `friends-count`: int64,
+    `statuses-count`: int64,
+    name: string,
+    `followers-count`: int64
+};
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    `sender-location`: point,
+    `send-time`: datetime,
+    `referred-topics`: {{ string }},
+    `message-text`: string,
+    countA: int64,
+    countB: int64
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(`sender-location`) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(`message-text`) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
new file mode 100644
index 0000000..cb2a156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
new file mode 100644
index 0000000..7d0f895
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select value count(*) from (
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where t1.countA /*+ indexnl */ = t2.tweetid
+    )}
+from  TweetMessages as t1
+where t1.countA > 0) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
new file mode 100644
index 0000000..c7685a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available secondary index in the index subtree.
+ *                In the probe side, this is an index-only plan since a secondary index-search will
+ *                fetch all fields that are required for the plan.
+ *                In the inner branch, this is also an index-only plan since a secondary index-search will
+ *                fetch all fields that are required for the branch.
+ *                All other variables will be propagated from the outer branch.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TwitterUserType as closed {
+    `screen-name`: string,
+    lang: string,
+    `friends-count`: int64,
+    `statuses-count`: int64,
+    name: string,
+    `followers-count`: int64
+};
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    `sender-location`: point,
+    `send-time`: datetime,
+    `referred-topics`: {{ string }},
+    `message-text`: string,
+    countA: int64,
+    countB: int64
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(`sender-location`) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(`message-text`) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
new file mode 100644
index 0000000..cb2a156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
new file mode 100644
index 0000000..6e2e769
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where t1.countA /*+ indexnl */ = t2.countB
+    )}
+from  TweetMessages as t1
+where t1.countA > 0) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
new file mode 100644
index 0000000..3e85829
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.1.ddl.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available primary index in the index subtree.
+ *                In the probe side, this is a non index-only plan since a secondary index-search can't cover
+ *                all fields that are required for the plan.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TwitterUserType as closed {
+    `screen-name`: string,
+    lang: string,
+    `friends-count`: int64,
+    `statuses-count`: int64,
+    name: string,
+    `followers-count`: int64
+};
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    `sender-location`: point,
+    `send-time`: datetime,
+    `referred-topics`: {{ string }},
+    `message-text`: string,
+    countA: int64,
+    countB: int64
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(`sender-location`) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(`message-text`) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
new file mode 100644
index 0000000..cb2a156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
new file mode 100644
index 0000000..312b329
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1/probe-sidx-btree-non-indexonly-plan-with-join-btree-pidx1.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select value count(*) from (
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where t1.countA /*+ indexnl */ = t2.tweetid
+    )}
+from  TweetMessages as t1
+where t1.countA > 0 and t1.countB < 10000) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
new file mode 100644
index 0000000..3d8f3e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.1.ddl.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test that left-outer-join may use an available secondary index in the index subtree.
+ *                In the probe side, this is a non index-only plan since a secondary index-search can't cover
+ *                all fields that are required for the plan.
+ *                In the inner branch, this is a index-only plan since a secondary index-search can cover
+ *                all fields that are required for that branch.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TwitterUserType as closed {
+    `screen-name`: string,
+    lang: string,
+    `friends-count`: int64,
+    `statuses-count`: int64,
+    name: string,
+    `followers-count`: int64
+};
+
+create type TweetMessageType as closed {
+    tweetid: int64,
+    user: TwitterUserType,
+    `sender-location`: point,
+    `send-time`: datetime,
+    `referred-topics`: {{ string }},
+    `message-text`: string,
+    countA: int64,
+    countB: int64
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
+
+create index twmSndLocIx on TweetMessages(`sender-location`) type rtree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index msgTextIx on TweetMessages(`message-text`) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
new file mode 100644
index 0000000..cb2a156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset TweetMessages
+using localfs
+(("path"="asterix_nc1://data/twitter/tw_for_indexleftouterjoin.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
new file mode 100644
index 0000000..2cae1d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-leftouterjoin/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan/probe-sidx-btree-non-indexonly-plan-with-join-btree-sidx1-indexonly-plan.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where t1.countA /*+ indexnl */ = t2.countB
+    )}
+from  TweetMessages as t1
+where t1.countA > 0 and t1.countB < 10000) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.sqlpp
new file mode 100644
index 0000000..57e8d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.sqlpp
new file mode 100644
index 0000000..bc705b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.sqlpp
new file mode 100644
index 0000000..ee9242c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-01/btree-secondary-composite-index-indexonly-plan-01.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {"pk":o.id, "sk":o.val1}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.sqlpp
new file mode 100644
index 0000000..57e8d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                                union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.sqlpp
new file mode 100644
index 0000000..867c708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.3.query.sqlpp
new file mode 100644
index 0000000..37c54e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-02/btree-secondary-composite-index-indexonly-plan-02.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use test;
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.sqlpp
new file mode 100644
index 0000000..4530d8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : This test is intended to verify that the secondary BTree index is
+ *                  : used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select   -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split -> select (the second condition) -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid_val1 on MyData(docid,val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.sqlpp
new file mode 100644
index 0000000..bc705b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.sqlpp
new file mode 100644
index 0000000..73c7632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-composite-index-indexonly-plan-03/btree-secondary-composite-index-indexonly-plan-03.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {"pk":o.id, "sk":o.docid, "sk2":o.val1}
+from MyData o
+where o.docid < 3 and o.val1 >= 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
new file mode 100644
index 0000000..c0e76e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.sqlpp
new file mode 100644
index 0000000..1094ca1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.10.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select o
+from MyData o
+where o.docid < 3
+order by o.id) a;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.sqlpp
new file mode 100644
index 0000000..bc705b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData2.json"),("format"="adm"));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.sqlpp
new file mode 100644
index 0000000..a4311fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.sqlpp
new file mode 100644
index 0000000..37c1d49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {"pk":o.id}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.sqlpp
new file mode 100644
index 0000000..46aca8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element {"sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.sqlpp
new file mode 100644
index 0000000..226d9d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.sqlpp
new file mode 100644
index 0000000..c97dc08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {"pk":o.id}
+from MyData o
+where o.docid < 3
+order by o.id) a;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.sqlpp
new file mode 100644
index 0000000..b7676d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.8.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {"sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.sqlpp
new file mode 100644
index 0000000..2db329a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-indexonly-plan-01/btree-secondary-index-indexonly-plan-01.9.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from (
+select element {"rec":o, "pk":o.id}
+from MyData o
+where o.docid < 3
+order by o.id) a;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
new file mode 100644
index 0000000..c2251fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/btree-secondary-index-non-indexonly-plan-01/btree-secondary-index-non-indexonly-plan-01.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we have multiple conditions that one index can cover.
+ *                  : Thus, index-only plan is not possible.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;


[02/16] asterixdb git commit: [ASTERIXDB-1972][COMP][RT][TX] index-only plan

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 7a24400..a081fb4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -41,6 +41,7 @@ import org.apache.asterix.common.metadata.LockList;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.api.IDataSourceAdapter;
@@ -432,7 +433,7 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
             boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes) throws AlgebricksException {
+            int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
         boolean isSecondary = true;
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), dataset.getDatasetName());
@@ -450,16 +451,34 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             primaryKeyFields[i] = i;
         }
 
-        ISearchOperationCallbackFactory searchCallbackFactory = dataset
-                .getSearchCallbackFactory(storageComponentProvider, theIndex, IndexOperation.SEARCH, primaryKeyFields);
+        int[] primaryKeyFieldsInSecondaryIndex = null;
+        byte[] successValueForIndexOnlyPlan = null;
+        byte[] failValueForIndexOnlyPlan = null;
+        boolean proceedIndexOnlyPlan = isIndexOnlyPlan && isSecondary;
+        if (proceedIndexOnlyPlan) {
+            int numSecondaryKeys = theIndex.getKeyFieldNames().size();
+            primaryKeyFieldsInSecondaryIndex = new int[numPrimaryKeys];
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                primaryKeyFieldsInSecondaryIndex[i] = i + numSecondaryKeys;
+            }
+            // Defines the return value from a secondary index search if this is an index-only plan.
+            failValueForIndexOnlyPlan = SerializerDeserializerUtil.computeByteArrayForIntValue(0);
+            successValueForIndexOnlyPlan = SerializerDeserializerUtil.computeByteArrayForIntValue(1);
+        }
+
+        ISearchOperationCallbackFactory searchCallbackFactory =
+                dataset.getSearchCallbackFactory(storageComponentProvider, theIndex, IndexOperation.SEARCH,
+                        primaryKeyFields, primaryKeyFieldsInSecondaryIndex, proceedIndexOnlyPlan);
         IStorageManager storageManager = getStorageComponentProvider().getStorageManager();
         IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(storageManager, spPc.first);
         BTreeSearchOperatorDescriptor btreeSearchOp;
+
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                     context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                    maxFilterFieldIndexes, propagateFilter);
+                    maxFilterFieldIndexes, propagateFilter, proceedIndexOnlyPlan, failValueForIndexOnlyPlan,
+                    successValueForIndexOnlyPlan);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -472,8 +491,8 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
-            int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes)
-            throws AlgebricksException {
+            int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+            boolean isIndexOnlyPlan) throws AlgebricksException {
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
         Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), indexName);
@@ -489,15 +508,38 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             primaryKeyFields[i] = i;
         }
 
-        ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                storageComponentProvider, secondaryIndex, IndexOperation.SEARCH, primaryKeyFields);
+        int[] primaryKeyFieldsInSecondaryIndex = null;
+        byte[] successValueForIndexOnlyPlan = null;
+        byte[] failValueForIndexOnlyPlan = null;
+        if (isIndexOnlyPlan) {
+            ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+            List<List<String>> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
+            List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+            Pair<IAType, Boolean> keyTypePair =
+                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
+            IAType keyType = keyTypePair.first;
+            int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+            int numNestedSecondaryKeyFields = numDimensions * 2;
+            primaryKeyFieldsInSecondaryIndex = new int[numPrimaryKeys];
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                primaryKeyFieldsInSecondaryIndex[i] = i + numNestedSecondaryKeyFields;
+            }
+            // Defines the return value from a secondary index search if this is an index-only plan.
+            failValueForIndexOnlyPlan = SerializerDeserializerUtil.computeByteArrayForIntValue(0);
+            successValueForIndexOnlyPlan = SerializerDeserializerUtil.computeByteArrayForIntValue(1);
+        }
+
+        ISearchOperationCallbackFactory searchCallbackFactory =
+                dataset.getSearchCallbackFactory(storageComponentProvider, secondaryIndex, IndexOperation.SEARCH,
+                        primaryKeyFields, primaryKeyFieldsInSecondaryIndex, isIndexOnlyPlan);
         RTreeSearchOperatorDescriptor rtreeSearchOp;
         IIndexDataflowHelperFactory indexDataflowHelperFactory =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
                     indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
-                    searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter);
+                    searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter,
+                    isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             // Create the operator
             rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 8cd7053..7981309 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -71,6 +71,7 @@ import org.apache.asterix.transaction.management.opcallbacks.LockThenSearchOpera
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexInstanctSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
@@ -540,20 +541,28 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     /**
      * Get search callback factory for this dataset with the passed index and operation
      *
+     * @param storageComponentProvider
+     *            storage component provider
      * @param index
      *            the index
      * @param op
      *            the operation this search is part of
      * @param primaryKeyFields
      *            the primary key fields indexes for locking purposes
+     * @param primaryKeyFieldsInSecondaryIndex
+     *            the primary key fields indexes in the given secondary index (used for index-only plan)
+     * @param proceedIndexOnlyPlan
+     *            the given plan is an index-only plan? (used for index-only plan)
      * @return
      *         an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
      * @throws AlgebricksException
      *             if the callback factory could not be created
      */
     public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
-            Index index, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
+            Index index, IndexOperation op, int[] primaryKeyFields, int[] primaryKeyFieldsInSecondaryIndex,
+            boolean proceedIndexOnlyPlan) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
+            /**
             /*
              * Due to the read-committed isolation level,
              * we may acquire very short duration lock(i.e., instant lock) for readers.
@@ -563,6 +572,12 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
                             storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE)
                     : new PrimaryIndexInstantSearchOperationCallbackFactory(getDatasetId(), primaryKeyFields,
                             storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE);
+        } else if (proceedIndexOnlyPlan) {
+            // Index-only plan case: we need to execute instantTryLock on PK during a secondary-index search.
+            // TODO: ResourceType is never used in the Callbackfactory. Should we keep it?
+            return new SecondaryIndexInstanctSearchOperationCallbackFactory(getDatasetId(),
+                    primaryKeyFieldsInSecondaryIndex, storageComponentProvider.getTransactionSubsystemProvider(),
+                    index.resourceType());
         } else if (index.getKeyFieldNames().isEmpty()) {
             // this is the case where the index is secondary primary index and locking is required
             // since the secondary primary index replaces the dataset index (which locks)
@@ -573,6 +588,27 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     }
 
     /**
+     * Get search callback factory for this dataset with the passed index and operation
+     *
+     * @param storageComponentProvider
+     *            storage component provider
+     * @param index
+     *            the index
+     * @param op
+     *            the operation this search is part of
+     * @param primaryKeyFields
+     *            the primary key fields indexes for locking purposes
+     * @return
+     *         an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
+     * @throws AlgebricksException
+     *             if the callback factory could not be created
+     */
+    public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
+            Index index, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
+        return getSearchCallbackFactory(storageComponentProvider, index, op, primaryKeyFields, null, false);
+    }
+
+    /**
      * Get the modification callback factory associated with this dataset, the passed index, and operation.
      *
      * @param index

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
index b0844ae..4ea2c13 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
@@ -22,11 +22,16 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public final class SerializerDeserializerUtil {
 
@@ -77,4 +82,20 @@ public final class SerializerDeserializerUtil {
             throw HyracksDataException.create(e);
         }
     }
+
+    /**
+     * Computes and returns the byte array for an integer value.
+     */
+    public static byte[] computeByteArrayForIntValue(int value) throws AlgebricksException {
+        ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
+        try {
+            AInt32 val = new AInt32(value);
+            SerializerDeserializerUtil.serializeTag(val, castBuffer.getDataOutput());
+            AInt32SerializerDeserializer.INSTANCE.serialize(val, castBuffer.getDataOutput());
+        } catch (HyracksDataException e) {
+            throw CompilationException.create(ErrorCode.CANNOT_SERIALIZE_A_VALUE, e);
+        }
+        return castBuffer.getByteArray();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
index 2a47337..67514a3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
@@ -28,6 +28,8 @@ import java.util.Map;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.types.ATypeTag;
@@ -193,12 +195,14 @@ public class ATypeHierarchy {
     // Get an AsterixConstantValue from a source Object
     public static AsterixConstantValue getAsterixConstantValueFromNumericTypeObject(IAObject sourceObject,
             ATypeTag targetTypeTag) throws HyracksDataException {
-        return getAsterixConstantValueFromNumericTypeObject(sourceObject, targetTypeTag, false);
+        return getAsterixConstantValueFromNumericTypeObject(sourceObject, targetTypeTag, false,
+                TypeCastingMathFunctionType.NONE);
     }
 
     // Get an AsterixConstantValue from a source Object
     public static AsterixConstantValue getAsterixConstantValueFromNumericTypeObject(IAObject sourceObject,
-            ATypeTag targetTypeTag, boolean strictDemote) throws HyracksDataException {
+            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         ATypeTag sourceTypeTag = sourceObject.getType().getTypeTag();
         if (sourceTypeTag == targetTypeTag) {
             return new AsterixConstantValue(sourceObject);
@@ -215,19 +219,19 @@ public class ATypeHierarchy {
             return null;
         }
 
-        IAObject targetObject = convertComputer.convertType(sourceObject);
+        IAObject targetObject = convertComputer.convertType(sourceObject, mathFunction);
         return new AsterixConstantValue(targetObject);
     }
 
     // Type Casting from source Object to an Object with Target type
     public static IAObject convertNumericTypeObject(IAObject sourceObject, ATypeTag targetTypeTag)
             throws HyracksDataException {
-        return convertNumericTypeObject(sourceObject, targetTypeTag, false);
+        return convertNumericTypeObject(sourceObject, targetTypeTag, false, TypeCastingMathFunctionType.NONE);
     }
 
     // Type Casting from source Object to an Object with Target type
-    public static IAObject convertNumericTypeObject(IAObject sourceObject, ATypeTag targetTypeTag, boolean strictDemote)
-            throws HyracksDataException {
+    public static IAObject convertNumericTypeObject(IAObject sourceObject, ATypeTag targetTypeTag, boolean strictDemote,
+            TypeCastingMathFunctionType mathFunction) throws HyracksDataException {
         ATypeTag sourceTypeTag = sourceObject.getType().getTypeTag();
         if (sourceTypeTag == targetTypeTag) {
             return sourceObject;
@@ -243,7 +247,7 @@ public class ATypeHierarchy {
             throw new RuntimeDataException(ErrorCode.TYPE_CONVERT, sourceTypeTag, targetTypeTag);
         }
 
-        return convertComputer.convertType(sourceObject);
+        return convertComputer.convertType(sourceObject, mathFunction);
     }
 
     // convert a numeric value in a byte array to the target type value
@@ -388,10 +392,46 @@ public class ATypeHierarchy {
         }
     }
 
+    /**
+     * Applies certain math function (e.g., ceil or floor) to a double value and returns that value.
+     */
+    public static double applyMathFunctionToDoubleValue(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws RuntimeDataException {
+        switch (mathFunction) {
+            case CEIL:
+                return Math.ceil(((ADouble) sourceObject).getDoubleValue());
+            case FLOOR:
+                return Math.floor(((ADouble) sourceObject).getDoubleValue());
+            default:
+                return ((ADouble) sourceObject).getDoubleValue();
+        }
+    }
+
+    /**
+     * Applies certain math function (e.g., ceil or floor) to a float value and returns that value.
+     */
+    public static float applyMathFunctionToFloatValue(IAObject sourceObject, TypeCastingMathFunctionType mathFunction) {
+        switch (mathFunction) {
+            case CEIL:
+                return (float) Math.ceil(((AFloat) sourceObject).getFloatValue());
+            case FLOOR:
+                return (float) Math.floor(((AFloat) sourceObject).getFloatValue());
+            default:
+                return ((AFloat) sourceObject).getFloatValue();
+        }
+    }
+
     public enum Domain {
         SPATIAL,
         NUMERIC,
         LIST,
         ANY
     }
+
+    // Type-casting mathFunction that will be used to type-cast a FLOAT or a DOUBLE value into an INTEGER value.
+    public enum TypeCastingMathFunctionType {
+        CEIL,
+        FLOOR,
+        NONE
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToFloatTypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToFloatTypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToFloatTypeConvertComputer.java
index 70aa283..361d06a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToFloatTypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToFloatTypeConvertComputer.java
@@ -27,6 +27,7 @@ import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
@@ -55,7 +56,12 @@ public class DoubleToFloatTypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        if (mathFunction != TypeCastingMathFunctionType.NONE) {
+            throw new RuntimeDataException(ErrorCode.INVALID_TYPE_CASTING_MATH_FUNCTION, mathFunction, ATypeTag.DOUBLE,
+                    ATypeTag.FLOAT);
+        }
         double sourceValue = ((ADouble) sourceObject).getDoubleValue();
         float targetValue = convert(sourceValue);
         return new AFloat(targetValue);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
index 1c69dc4..85d71a8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
@@ -55,8 +55,9 @@ public class DoubleToInt16TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        double sourceValue = ((ADouble) sourceObject).getDoubleValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        double sourceValue = ATypeHierarchy.applyMathFunctionToDoubleValue(sourceObject, mathFunction);
         short targetValue = convert(sourceValue);
         return new AInt16(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
index 7d2f121..8bd9f3e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
@@ -59,8 +59,9 @@ public class DoubleToInt32TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        double sourceValue = ((ADouble) sourceObject).getDoubleValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        double sourceValue = ATypeHierarchy.applyMathFunctionToDoubleValue(sourceObject, mathFunction);
         int targetValue = convert(sourceValue);
         return new AInt32(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
index 6098afa..cb9a6bc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
@@ -63,8 +63,9 @@ public class DoubleToInt64TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        double sourceValue = ((ADouble) sourceObject).getDoubleValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        double sourceValue = ATypeHierarchy.applyMathFunctionToDoubleValue(sourceObject, mathFunction);
         long targetValue = convert(sourceValue);
         return new AInt64(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
index bb1ccc3..69934b5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
@@ -55,8 +55,9 @@ public class DoubleToInt8TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        double sourceValue = ((ADouble) sourceObject).getDoubleValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        double sourceValue = ATypeHierarchy.applyMathFunctionToDoubleValue(sourceObject, mathFunction);
         byte targetValue = convert(sourceValue);
         return new AInt8(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToDoubleTypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToDoubleTypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToDoubleTypeConvertComputer.java
index 548768e..7f780ce 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToDoubleTypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToDoubleTypeConvertComputer.java
@@ -21,10 +21,13 @@ package org.apache.asterix.om.types.hierachy;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
@@ -48,7 +51,12 @@ public class FloatToDoubleTypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        if (mathFunction != TypeCastingMathFunctionType.NONE) {
+            throw new RuntimeDataException(ErrorCode.INVALID_TYPE_CASTING_MATH_FUNCTION, mathFunction, ATypeTag.FLOAT,
+                    ATypeTag.DOUBLE);
+        }
         double targetValue = ((AFloat) sourceObject).getFloatValue();
         return new ADouble(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
index ed570fc..4906c56 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 
@@ -55,8 +55,9 @@ public class FloatToInt16TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        float sourceValue = ((AFloat) sourceObject).getFloatValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        float sourceValue = ATypeHierarchy.applyMathFunctionToFloatValue(sourceObject, mathFunction);
         short targetValue = convert(sourceValue);
         return new AInt16(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
index 2c81c8e..366278d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 
@@ -59,8 +59,9 @@ public class FloatToInt32TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        float sourceValue = ((AFloat) sourceObject).getFloatValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        float sourceValue = ATypeHierarchy.applyMathFunctionToFloatValue(sourceObject, mathFunction);
         int targetValue = convert(sourceValue);
         return new AInt32(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
index 9180785..291cab6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 
@@ -54,8 +54,9 @@ public class FloatToInt64TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        float sourceValue = ((AFloat) sourceObject).getFloatValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        float sourceValue = ATypeHierarchy.applyMathFunctionToFloatValue(sourceObject, mathFunction);
         long targetValue = convert(sourceValue);
         return new AInt64(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
index 43a5e76..30f19c3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
@@ -23,10 +23,10 @@ import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 
@@ -55,8 +55,9 @@ public class FloatToInt8TypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
-        float sourceValue = ((AFloat) sourceObject).getFloatValue();
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
+        float sourceValue = ATypeHierarchy.applyMathFunctionToFloatValue(sourceObject, mathFunction);
         byte targetValue = convert(sourceValue);
         return new AInt8(targetValue);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ITypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ITypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ITypeConvertComputer.java
index a13878d..625a52c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ITypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ITypeConvertComputer.java
@@ -22,6 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ITypeConvertComputer {
@@ -30,5 +31,6 @@ public interface ITypeConvertComputer {
     void convertType(byte[] data, int start, int length, DataOutput out) throws IOException;
 
     // promote or demote a type to a different type
-    IAObject convertType(IAObject sourceObject) throws HyracksDataException;
+    // mathFunction (e.g., ceil or floor) is used to convert a real value into an integer value.
+    IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToDoubleTypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToDoubleTypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToDoubleTypeConvertComputer.java
index 55bba87..f0da652 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToDoubleTypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToDoubleTypeConvertComputer.java
@@ -31,6 +31,7 @@ import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.BytePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -73,7 +74,8 @@ public class IntegerToDoubleTypeConvertComputer implements ITypeConvertComputer
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         long val;
         switch (sourceObject.getType().getTypeTag()) {
             case TINYINT:

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToFloatTypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToFloatTypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToFloatTypeConvertComputer.java
index 4aa4749..5ca3a51 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToFloatTypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToFloatTypeConvertComputer.java
@@ -31,6 +31,7 @@ import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.BytePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -75,7 +76,8 @@ public class IntegerToFloatTypeConvertComputer implements ITypeConvertComputer {
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         float val;
         switch (sourceObject.getType().getTypeTag()) {
             case TINYINT:

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt16TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt16TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt16TypeConvertComputer.java
index 86363fc..f3dbb59 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt16TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt16TypeConvertComputer.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class IntegerToInt16TypeConvertComputer extends AbstractIntegerTypeConvertComputer {
@@ -46,7 +47,8 @@ public class IntegerToInt16TypeConvertComputer extends AbstractIntegerTypeConver
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         return convertIntegerType(sourceObject, ATypeTag.SMALLINT);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt32TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt32TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt32TypeConvertComputer.java
index 73f16fa..3ebf240 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt32TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt32TypeConvertComputer.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class IntegerToInt32TypeConvertComputer extends AbstractIntegerTypeConvertComputer {
@@ -46,7 +47,8 @@ public class IntegerToInt32TypeConvertComputer extends AbstractIntegerTypeConver
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         return convertIntegerType(sourceObject, ATypeTag.INTEGER);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt64TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt64TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt64TypeConvertComputer.java
index bfe1625..a3cfa99 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt64TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt64TypeConvertComputer.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class IntegerToInt64TypeConvertComputer extends AbstractIntegerTypeConvertComputer {
@@ -43,7 +44,8 @@ public class IntegerToInt64TypeConvertComputer extends AbstractIntegerTypeConver
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         return convertIntegerType(sourceObject, ATypeTag.BIGINT);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt8TypeConvertComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt8TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt8TypeConvertComputer.java
index 32e3d00..61684d6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt8TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/IntegerToInt8TypeConvertComputer.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class IntegerToInt8TypeConvertComputer extends AbstractIntegerTypeConvertComputer {
@@ -45,7 +46,8 @@ public class IntegerToInt8TypeConvertComputer extends AbstractIntegerTypeConvert
     }
 
     @Override
-    public IAObject convertType(IAObject sourceObject) throws HyracksDataException {
+    public IAObject convertType(IAObject sourceObject, TypeCastingMathFunctionType mathFunction)
+            throws HyracksDataException {
         return convertIntegerType(sourceObject, ATypeTag.TINYINT);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstanctSearchOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstanctSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstanctSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..ec6fdbe
--- /dev/null
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstanctSearchOperationCallbackFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.transaction.management.opcallbacks;
+
+import org.apache.asterix.common.api.IJobEventListenerFactory;
+import org.apache.asterix.common.context.ITransactionSubsystemProvider;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.AbstractOperationCallbackFactory;
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IJobletEventListenerFactory;
+import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
+
+/**
+ * Instant-search-operation-callback class for the index-only plan. The purpose of this callback is executing an instant
+ * tryLock on PK during a secondary index-search.
+ *
+ */
+public class SecondaryIndexInstanctSearchOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements ISearchOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public SecondaryIndexInstanctSearchOperationCallbackFactory(int datasetId, int[] entityIdFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+        super(datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+    }
+
+    @Override
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx,
+            IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
+        try {
+            // If the plan is an index-only query plan, we need to try to get an instant try lock on PK.
+            // If an instant tryLock on PK fails, we do not attempt to do a lock since the operations
+            // will be dealt with in the operators after the given secondary-index search.
+            ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+            IJobletEventListenerFactory fact = ctx.getJobletContext().getJobletEventListenerFactory();
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager()
+                    .getTransactionContext(((IJobEventListenerFactory) fact).getTxnId(datasetId));
+            return new SecondaryIndexInstantSearchOperationCallback(new DatasetId(datasetId), resourceId,
+                    primaryKeyFields, txnSubsystem.getLockManager(), txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstantSearchOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstantSearchOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstantSearchOperationCallback.java
new file mode 100644
index 0000000..2f2a1c5
--- /dev/null
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexInstantSearchOperationCallback.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ *     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.transaction.management.opcallbacks;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.AbstractOperationCallback;
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
+
+/**
+ * Assumes LSM-BTrees as primary indexes. Implements try/locking and unlocking on primary keys.
+ * This Callback method tries to get a lock on PK (instantTryLock).
+ * If it fails, this callback does nothing since its purpose is to attempt to get an instant lock
+ * and get the result of it. This operation callback is used in an index-only plan.
+ */
+public class SecondaryIndexInstantSearchOperationCallback extends AbstractOperationCallback
+        implements ISearchOperationCallback {
+
+    public SecondaryIndexInstantSearchOperationCallback(DatasetId datasetId, long resourceId, int[] entityIdFields,
+            ILockManager lockManager, ITransactionContext txnCtx) {
+        super(datasetId, resourceId, entityIdFields, txnCtx, lockManager);
+    }
+
+    @Override
+    public void before(ITupleReference tuple) throws HyracksDataException {
+        // This will not be used for a modification operation.
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            return lockManager.instantTryLock(datasetId, pkHash, LockMode.S, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) throws HyracksDataException {
+        // No reconciled required since the purpose is instantTryLock on PK.
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) throws HyracksDataException {
+        // No cancel required since reconcile operation is empty.
+    }
+
+    @Override
+    public void complete(ITupleReference tuple) throws HyracksDataException {
+        // No cancel required since reconcile operation is empty.
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Quadruple.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Quadruple.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Quadruple.java
new file mode 100644
index 0000000..6e393bb
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Quadruple.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ *     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.algebricks.common.utils;
+
+import java.util.Objects;
+
+public class Quadruple<T1, T2, T3, T4> {
+    private T1 first;
+    private T2 second;
+    private T3 third;
+    private T4 fourth;
+
+    public Quadruple(T1 first, T2 second, T3 third, T4 fourth) {
+        this.first = first;
+        this.second = second;
+        this.third = third;
+        this.fourth = fourth;
+    }
+
+    public void setFirst(T1 first) {
+        this.first = first;
+    }
+
+    public void setSecond(T2 second) {
+        this.second = second;
+    }
+
+    public void setThird(T3 third) {
+        this.third = third;
+    }
+
+    public void setFourth(T4 fourth) {
+        this.fourth = fourth;
+    }
+
+    public T1 getFirst() {
+        return this.first;
+    }
+
+    public T2 getSecond() {
+        return this.second;
+    }
+
+    public T3 getThird() {
+        return this.third;
+    }
+
+    public T4 getFourth() {
+        return this.fourth;
+    }
+
+    @Override
+    public String toString() {
+        return first + "," + second + ", " + third + ", " + fourth;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(first, second, third, fourth);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof Quadruple<?, ?, ?, ?>)) {
+            return false;
+        }
+        Quadruple<?, ?, ?, ?> quadruple = (Quadruple<?, ?, ?, ?>) o;
+        return Objects.equals(first, quadruple.first) && Objects.equals(second, quadruple.second)
+                && Objects.equals(third, quadruple.third) && Objects.equals(fourth, quadruple.fourth);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index a8e6b44..c2d2cfa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -36,6 +36,8 @@ public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
     protected List<LogicalVariable> maxFilterVars;
 
     protected boolean propagateIndexFilter;
+    // Used when the result of a searchCallBack.proceed() is required afterwards.
+    protected boolean generateSearchCallBackProceedResultVar;
 
     public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
@@ -44,15 +46,20 @@ public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
         this.variableTypes = variableTypes;
         this.propagateInput = propagateInput;
         this.propagateIndexFilter = false;
+        this.generateSearchCallBackProceedResultVar = false;
     }
 
     @Override
     public List<LogicalVariable> getScanVariables() {
+        // An additional variable - generateSearchCallBackProceedResultVar should not be returned.
+        int excludeVarCount = 0;
         if (propagateIndexFilter) {
-            return variables.subList(0, variables.size() - 2);
-        } else {
-            return variables;
+            excludeVarCount += 2;
+        }
+        if (generateSearchCallBackProceedResultVar) {
+            excludeVarCount++;
         }
+        return excludeVarCount > 0 ? variables.subList(0, variables.size() - excludeVarCount) : variables;
     }
 
     public List<Object> getVariableTypes() {
@@ -133,4 +140,18 @@ public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
             return null;
         }
     }
+
+    /**
+     * Sets the variable to tell whether the result of a searchCallBack.proceed() is required.
+     * If this variable is set to true, the last variable in the variables list should contain
+     * the result of a searchCallBack.proceed().
+     */
+    public void setGenerateCallBackProceedResultVar(boolean generateCallBackProceedResultVar) {
+        this.generateSearchCallBackProceedResultVar = generateCallBackProceedResultVar;
+    }
+
+    public boolean getGenerateCallBackProceedResultVar() {
+        return this.generateSearchCallBackProceedResultVar;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
index a17e2a0..817eeaf 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.operators.logical;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -60,7 +61,7 @@ public class LeftOuterUnnestMapOperator extends AbstractUnnestMapOperator {
         // Propagates all input variables that come from the outer branch.
         PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
 
-        env.getCorrelatedMissableVariableLists().add(variables);
+        env.getCorrelatedMissableVariableLists().add(new ArrayList<>(variables));
 
         // For the variables from the inner branch, the output type is the union
         // of (original type + null).

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 0ad3fea..70f19c1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -369,7 +369,7 @@ public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPr
     @Override
     public Void visitSplitOperator(SplitOperator op, Integer indent) throws AlgebricksException {
         Mutable<ILogicalExpression> branchingExpression = op.getBranchingExpression();
-        addIndent(indent).append("split " + branchingExpression.getValue().accept(exprVisitor, indent));
+        addIndent(indent).append("split (" + branchingExpression.getValue().accept(exprVisitor, indent) + ")");
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 4283198..1076eb5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -524,7 +524,8 @@ public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperat
     public Void visitSplitOperator(SplitOperator op, Integer indent) throws AlgebricksException {
         Mutable<ILogicalExpression> branchingExpression = op.getBranchingExpression();
         addIndent(indent).append("\"operator\": \"split\",\n");
-        addIndent(indent).append("\"" + branchingExpression.getValue().accept(exprVisitor, indent) + "\"");
+        addIndent(indent).append("\"expressions\": \""
+                + branchingExpression.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index e0d806d..a5cd5fa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -59,9 +59,15 @@ public class OperatorPropertiesUtil {
         return true;
     }
 
-    // Obs: doesn't return expected result for op. with nested plans.
-    private static void getFreeVariablesInOp(ILogicalOperator op, Set<LogicalVariable> freeVars)
+    /**
+     * Adds the free variables of the operator to the given set.
+     *
+     * @param op
+     * @param freeVars
+     */
+    public static void getFreeVariablesInOp(ILogicalOperator op, Set<LogicalVariable> freeVars)
             throws AlgebricksException {
+        // Obs: doesn't return expected result for op. with nested plans.
         VariableUtilities.getUsedVariables(op, freeVars);
         HashSet<LogicalVariable> produced = new HashSet<>();
         VariableUtilities.getProducedVariables(op, produced);
@@ -248,6 +254,16 @@ public class OperatorPropertiesUtil {
         }
     }
 
+    /**
+     * Recursively visits all descendants of the given operator and
+     * (re)computes and sets a type environment for each operator.
+     *
+     * @param r
+     *            a mutable logical operator
+     * @param context
+     *            optimization context
+     * @throws AlgebricksException
+     */
     public static void typeOpRec(Mutable<ILogicalOperator> r, IOptimizationContext context) throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) r.getValue();
         for (Mutable<ILogicalOperator> i : op.getInputs()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 923ffb5..4c2d910 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -37,6 +37,7 @@ import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
@@ -466,7 +467,7 @@ public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
         int outputIndex = 0;
         if (opRef.getValue().getOperatorTag() == LogicalOperatorTag.REPLICATE
                 || opRef.getValue().getOperatorTag() == LogicalOperatorTag.SPLIT) {
-            ReplicateOperator rop = (ReplicateOperator) opRef.getValue();
+            AbstractReplicateOperator rop = (AbstractReplicateOperator) opRef.getValue();
             List<Mutable<ILogicalOperator>> outputs = rop.getOutputs();
             for (outputIndex = 0; outputIndex < outputs.size(); outputIndex++) {
                 if (outputs.get(outputIndex).equals(parentRef)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
index 44a7b57..a724014 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
@@ -29,14 +29,12 @@ import java.util.Set;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -177,16 +175,8 @@ public class IntroduceProjectsRule implements IAlgebraicRewriteRule {
             ProjectOperator projectOp = (ProjectOperator) op;
             List<LogicalVariable> projectVarsTemp = projectOp.getVariables();
             if (liveVars.size() == projectVarsTemp.size() && liveVars.containsAll(projectVarsTemp)) {
-                boolean eliminateProject = true;
-                // For UnionAll the variables must also be in exactly the correct order.
-                if (parentOp.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
-                    eliminateProject =
-                            canEliminateProjectBelowUnion((UnionAllOperator) parentOp, projectOp, parentInputIndex);
-                }
-                if (eliminateProject) {
-                    // The existing project has become useless. Remove it.
-                    parentOp.getInputs().get(parentInputIndex).setValue(op.getInputs().get(0).getValue());
-                }
+                // The existing project has become useless. Remove it.
+                parentOp.getInputs().get(parentInputIndex).setValue(op.getInputs().get(0).getValue());
             }
         }
 
@@ -196,23 +186,4 @@ public class IntroduceProjectsRule implements IAlgebraicRewriteRule {
         return modified;
     }
 
-    private boolean canEliminateProjectBelowUnion(UnionAllOperator unionOp, ProjectOperator projectOp,
-            int unionInputIndex) throws AlgebricksException {
-        List<LogicalVariable> orderedLiveVars = new ArrayList<>();
-        VariableUtilities.getLiveVariables(projectOp.getInputs().get(0).getValue(), orderedLiveVars);
-        int numVars = orderedLiveVars.size();
-        for (int i = 0; i < numVars; i++) {
-            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple = unionOp.getVariableMappings().get(i);
-            if (unionInputIndex == 0) {
-                if (varTriple.first != orderedLiveVars.get(i)) {
-                    return false;
-                }
-            } else {
-                if (varTriple.second != orderedLiveVars.get(i)) {
-                    return false;
-                }
-            }
-        }
-        return true;
-    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
index e197814..062bb2f 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -146,12 +146,15 @@ public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule
         Set<LogicalVariable> assignVarsSetForThisOp = removeAssignVarFromConsideration(opRef);
 
         while (removeFromAssigns(op, assignVarsSetForThisOp, context) == 0) {
-            if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+            // UnionAllOperator cannot be removed since it has two branches.
+            if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE
+                    || op.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
                 break;
             }
             op = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
             opRef.setValue(op);
             assignVarsSetForThisOp = removeAssignVarFromConsideration(opRef);
+            isTransformed = true;
         }
 
         Iterator<Mutable<ILogicalOperator>> childIter = op.getInputs().iterator();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 8b47171..ad2d6f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -140,6 +140,8 @@ public class ErrorCode {
     public static final int INDEX_DOES_NOT_EXIST = 104;
     public static final int CANNOT_DROP_IN_USE_INDEX = 105;
     public static final int CANNOT_DEACTIVATE_PINNED_BLOOM_FILTER = 106;
+    public static final int PREDICATE_CANNOT_BE_NULL = 107;
+    public static final int FULLTEXT_ONLY_EXECUTABLE_FOR_STRING_OR_LIST = 108;
 
     // Compilation error codes.
     public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index a27a736..c3945a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -123,5 +123,7 @@
 104 = Index does not exist
 105 = Cannot drop in-use index (%1$s)
 106 = Failed to deactivate the bloom filter since it is pinned by other users
+107 = The given search predicate can't be null.
+108 = Full-text search can be only executed on STRING or (UN)ORDERED LIST.
 
 10000 = The given rule collection %1$s is not an instance of the List class.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3c23574/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 02bb20e..49eca6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -37,20 +37,35 @@ public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
     protected final int[] highKeyFields;
     protected final boolean lowKeyInclusive;
     protected final boolean highKeyInclusive;
-    protected final int[] minFilterFieldIndexes;
-    protected final int[] maxFilterFieldIndexes;
+    private final int[] minFilterFieldIndexes;
+    private final int[] maxFilterFieldIndexes;
     protected final IIndexDataflowHelperFactory indexHelperFactory;
     protected final boolean retainInput;
     protected final boolean retainMissing;
     protected final IMissingWriterFactory missingWriterFactory;
     protected final ISearchOperationCallbackFactory searchCallbackFactory;
     protected final boolean appendIndexFilter;
+    protected boolean appendOpCallbackProceedResult;
+    protected byte[] searchCallbackProceedResultFalseValue;
+    protected byte[] searchCallbackProceedResultTrueValue;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+        this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
+                maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+    }
+
+    public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
+            int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+            IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
+            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -65,6 +80,9 @@ public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
         this.outRecDescs[0] = outRecDesc;
+        this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
+        this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
+        this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
     }
 
     @Override
@@ -73,6 +91,9 @@ public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperato
         return new BTreeSearchOperatorNodePushable(ctx, partition,
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                searchCallbackProceedResultTrueValue);
     }
+
 }