You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2018/12/09 22:57:54 UTC

[1/6] asterixdb git commit: [ASTERIXDB-2478][NET] Calculate Buffer Remaining Before Reusing It

Repository: asterixdb
Updated Branches:
  refs/heads/master 43e0b15bc -> f184a1e7b


[ASTERIXDB-2478][NET] Calculate Buffer Remaining Before Reusing It

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

Details:
- When recycling a buffer, calculate the buffer remaining before
  releasing it for reuse to prevent other threads from changing
  its remaining.
- Add test case.

Change-Id: Icca3284feae800dd6c37694bdefec3516cd4c506
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3036
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: 821c072313633f1ef052b167fd0eb679bb7c6e52
Parents: 7311b03
Author: Murtadha Hubail <mh...@apache.org>
Authored: Tue Nov 20 14:05:53 2018 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Wed Nov 28 12:43:39 2018 -0800

----------------------------------------------------------------------
 hyracks-fullstack/hyracks/hyracks-net/pom.xml   |   6 +
 .../muxdemux/FullFrameChannelReadInterface.java |   4 +-
 .../FullFrameChannelReadInterfaceTest.java      | 180 +++++++++++++++++++
 3 files changed, 188 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/821c0723/hyracks-fullstack/hyracks/hyracks-net/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 1040e81..4ca20ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -60,5 +60,11 @@
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>2.0.2-beta</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/821c0723/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
index 32bf77e..3ba8627 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
@@ -36,7 +36,7 @@ public class FullFrameChannelReadInterface extends AbstractChannelReadInterface
     private final BlockingDeque<ByteBuffer> riEmptyStack;
     private final IChannelControlBlock ccb;
 
-    FullFrameChannelReadInterface(IChannelControlBlock ccb) {
+    public FullFrameChannelReadInterface(IChannelControlBlock ccb) {
         this.ccb = ccb;
         riEmptyStack = new LinkedBlockingDeque<>();
         credits = 0;
@@ -45,8 +45,8 @@ public class FullFrameChannelReadInterface extends AbstractChannelReadInterface
             if (ccb.isRemotelyClosed()) {
                 return;
             }
-            riEmptyStack.push(buffer);
             final int delta = buffer.remaining();
+            riEmptyStack.push(buffer);
             ccb.addPendingCredits(delta);
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/821c0723/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
new file mode 100644
index 0000000..f9a610c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.net.tests;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.comm.IBufferFactory;
+import org.apache.hyracks.api.comm.IChannelControlBlock;
+import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
+import org.apache.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import org.apache.hyracks.net.protocols.muxdemux.FullFrameChannelReadInterface;
+import org.apache.hyracks.util.StorageUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+@RunWith(Parameterized.class)
+public class FullFrameChannelReadInterfaceTest {
+
+    private static final int TEST_RUNS = 100;
+    private static final int RECEIVER_BUFFER_COUNT = 50;
+    private static int FRAMES_TO_READ_COUNT = 10000;
+    private static final int FRAME_SIZE = StorageUtil.getIntSizeInBytes(32, StorageUtil.StorageUnit.KILOBYTE);
+    private static final int EXPECTED_CHANNEL_CREDIT = FRAME_SIZE * RECEIVER_BUFFER_COUNT;
+
+    @Parameterized.Parameters
+    public static Object[][] data() {
+        return new Object[TEST_RUNS][0];
+    }
+
+    @Test
+    public void bufferRecycleTest() throws Exception {
+        final AtomicInteger channelCredit = new AtomicInteger();
+        final IChannelControlBlock ccb = mockChannelControlBlock(channelCredit);
+        final ReadBufferFactory bufferFactory = new ReadBufferFactory(RECEIVER_BUFFER_COUNT, FRAME_SIZE);
+        final FullFrameChannelReadInterface readInterface = new FullFrameChannelReadInterface(ccb);
+        final LinkedBlockingDeque<ByteBuffer> fullBufferQ = new LinkedBlockingDeque<>();
+        readInterface.setFullBufferAcceptor(new ReadFullBufferAcceptor(fullBufferQ));
+        readInterface.setBufferFactory(bufferFactory, RECEIVER_BUFFER_COUNT, FRAME_SIZE);
+        Assert.assertEquals(EXPECTED_CHANNEL_CREDIT, channelCredit.get());
+        final SocketChannel socketChannel = mockSocketChannel(ccb);
+        final Thread networkFrameReader = new Thread(() -> {
+            try {
+                int framesRead = FRAMES_TO_READ_COUNT;
+                while (framesRead > 0) {
+                    while (channelCredit.get() == 0) {
+                        synchronized (channelCredit) {
+                            channelCredit.wait(10000);
+                            if (channelCredit.get() == 0) {
+                                System.err.println("Sender doesn't have any write credit");
+                                System.exit(1);
+                            }
+                        }
+                    }
+                    readInterface.read(socketChannel, FRAME_SIZE);
+                    framesRead--;
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+
+        final Thread frameProcessor = new Thread(() -> {
+            int framesProcessed = 0;
+            try {
+                while (true) {
+                    final ByteBuffer fullFrame = fullBufferQ.take();
+                    fullFrame.clear();
+                    readInterface.getEmptyBufferAcceptor().accept(fullFrame);
+                    framesProcessed++;
+                    if (framesProcessed == FRAMES_TO_READ_COUNT) {
+                        return;
+                    }
+                }
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        });
+        networkFrameReader.start();
+        frameProcessor.start();
+        networkFrameReader.join();
+        frameProcessor.join();
+        if (channelCredit.get() != EXPECTED_CHANNEL_CREDIT) {
+            System.err
+                    .println("Expected channel credit " + EXPECTED_CHANNEL_CREDIT + " , found " + channelCredit.get());
+            System.exit(1);
+        }
+    }
+
+    private IChannelControlBlock mockChannelControlBlock(AtomicInteger credit) {
+        final ChannelControlBlock ccb = Mockito.mock(ChannelControlBlock.class);
+        Mockito.when(ccb.isRemotelyClosed()).thenReturn(false);
+        Mockito.doAnswer(invocation -> {
+            final Integer delta = invocation.getArgumentAt(0, Integer.class);
+            credit.addAndGet(delta);
+            synchronized (credit) {
+                credit.notifyAll();
+            }
+            return null;
+        }).when(ccb).addPendingCredits(Mockito.anyInt());
+        return ccb;
+    }
+
+    private SocketChannel mockSocketChannel(IChannelControlBlock ccb) throws IOException {
+        final SocketChannel sc = Mockito.mock(SocketChannel.class);
+        Mockito.when(sc.read(Mockito.any(ByteBuffer.class))).thenAnswer(invocation -> {
+            ccb.addPendingCredits(-FRAME_SIZE);
+            final ByteBuffer buffer = invocation.getArgumentAt(0, ByteBuffer.class);
+            while (buffer.hasRemaining()) {
+                buffer.put((byte) 0);
+            }
+            return FRAME_SIZE;
+        });
+        return sc;
+    }
+
+    private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+        private final BlockingQueue<ByteBuffer> fullBufferQ;
+
+        ReadFullBufferAcceptor(BlockingQueue<ByteBuffer> fullBuffer) {
+            this.fullBufferQ = fullBuffer;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            fullBufferQ.add(buffer);
+        }
+
+        @Override
+        public void close() {
+        }
+
+        @Override
+        public void error(int ecode) {
+        }
+    }
+
+    public class ReadBufferFactory implements IBufferFactory {
+        private final int limit;
+        private final int frameSize;
+        private int counter = 0;
+
+        ReadBufferFactory(int limit, int frameSize) {
+            this.limit = limit;
+            this.frameSize = frameSize;
+        }
+
+        @Override
+        public ByteBuffer createBuffer() {
+            if (counter >= limit) {
+                throw new IllegalStateException("Buffer limit exceeded");
+            }
+            counter++;
+            return ByteBuffer.allocate(frameSize);
+        }
+    }
+}
\ No newline at end of file


[4/6] asterixdb git commit: [NO ISSUE][IDX][COMP] Creating an open index on non-declared fields

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
new file mode 100644
index 0000000..9c6b8d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$34(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
new file mode 100644
index 0000000..00280ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- INTERSECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$33(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|
+                                    -- STABLE_SORT [$$37(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp
new file mode 100644
index 0000000..639fbd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp
@@ -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  : Index join for nested composite non-enforced indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp
new file mode 100644
index 0000000..0215367
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.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.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known. Promotable numeric type.
+  * Result       : Success
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_double(t1.nested.c_i8) /*+ indexnl */ = t2.nested.c_i8
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp
new file mode 100644
index 0000000..10fa158
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen1 ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "extra": 99
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "extra": 98
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "extra": 97
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "extra": 996
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "extra": 9
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "extra": 91
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3,
+        "extra": 99
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8,
+        "extra": 99
+    }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp
new file mode 100644
index 0000000..f3d6e8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen2 ({
+    "c_id": 101,
+    "nested": {
+        "c_x": 101,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "extra": 97
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 102,
+    "nested": {
+        "c_x": 102,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "extra": 96
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 103,
+    "nested": {
+        "c_x": 103,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "extra": 95
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 104,
+    "nested": {
+        "c_x": 104,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "extra": 94
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 105,
+    "nested": {
+        "c_x": 105,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "extra": 93
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 106,
+    "nested": {
+        "c_x": 106,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "extra": 91
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 107,
+    "nested": {
+        "c_x": 107,
+        "c_s": "world",
+        "c_i64": 3,
+        "extra": 97
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 108,
+    "nested": {
+        "c_x": 108,
+        "extra": 97
+    }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp
new file mode 100644
index 0000000..c8be482
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp
@@ -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.
+ */
+
+use test;
+
+create index idx_t1_s on TestOpen1(nested.c_s:string, nested.extra: int);
+
+create index idx_t1_i64 on TestOpen1(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t1_i8 on TestOpen1(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t1_d on TestOpen1(nested.c_d:double, nested.extra: int);
+
+/********************************************/
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp
new file mode 100644
index 0000000..1301e74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  * Description  : No index join because there's no hint, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp
new file mode 100644
index 0000000..a91e785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  * Description  : No index join because the probe type is unknown, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp
new file mode 100644
index 0000000..a3c2a88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  * Description  : No index join because there's no hint, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp
new file mode 100644
index 0000000..f8a990e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known (string), non-enforced nested composite
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp
new file mode 100644
index 0000000..40f27ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known (bigint), non-enforced nested composite
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.nested.c_i64) /*+ indexnl */ = t2.nested.c_i64
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp
new file mode 100644
index 0000000..2d26551
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.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, DBLP and CSX, based on their authors.
+ *                  CSX has a nested composite open index, and given the 'indexnl' hint 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.DBLPTypetmp as
+ closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXTypetmp as
+ closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as {
+  id: int
+};
+
+create type test.CSXType as {
+  id: int
+};
+
+create  dataset DBLPtmp(DBLPTypetmp) primary key id;
+
+create  dataset CSXtmp(CSXTypetmp) primary key id;
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp
new file mode 100644
index 0000000..b6f719a
Binary files /dev/null and b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp differ

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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp
new file mode 100644
index 0000000..6ec0578
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.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 {'aid':a.nested.id,'bid':b.nested.id,'authors':a.nested.authors}
+from  DBLP as a, CSX as b
+where to_string(a.nested.authors) /*+ indexnl */  = b.nested.authors
+order by a.nested.id, b.nested.id;
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp
new file mode 100644
index 0000000..9c7b4e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Index selection for composite nested non-enforced indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp
new file mode 100644
index 0000000..7d4c7da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_d >= 3.25
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp
new file mode 100644
index 0000000..e8ff5b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 499 and t.nested.c_i8 < 99999
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp
new file mode 100644
index 0000000..07298b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.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.
+ */
+drop dataverse test if exists;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp
new file mode 100644
index 0000000..c4046a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8,
+        "c_extra": "yes"
+    }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp
new file mode 100644
index 0000000..5a3d25a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_s on TestOpen(nested.c_s:string, nested.c_extra: string);
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.c_extra: string);
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.c_extra: string);
+
+create index idx_d on TestOpen(nested.c_d:double, nested.c_extra: string);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp
new file mode 100644
index 0000000..2462bda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp
new file mode 100644
index 0000000..0b59642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp
new file mode 100644
index 0000000..99a596f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp
new file mode 100644
index 0000000..ce16148
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2.0
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp
new file mode 100644
index 0000000..c0b4563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp
new file mode 100644
index 0000000..c5cc2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2.5
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp
new file mode 100644
index 0000000..079b5ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Index selection for composite nested non-enforced indexes multiple indexes match.
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp
new file mode 100644
index 0000000..2ff5578
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "c_x": 1,
+    "nested": {
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8
+    }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp
new file mode 100644
index 0000000..85928cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_2 on TestOpen(nested.c_s:string, nested.c_i64:int64);
+
+create index idx_1 on TestOpen(nested.c_s:string, nested.c_d:double);
+
+create index idx_3 on TestOpen(nested.c_i64:int64);
+
+create index idx_4 on TestOpen(nested.c_i8:int8);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp
new file mode 100644
index 0000000..2462bda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp
new file mode 100644
index 0000000..b98ee96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.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 t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2 and t.nested.c_i64 < 3
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp
new file mode 100644
index 0000000..abf2843
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp
@@ -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  : Multiple nested composite non-enforced indexes on the same field, created before data loading
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type GleambookUserType as { id: int };
+
+create dataset GleambookUsers(GleambookUserType) primary key id;
+
+create index gbUserSinceIdxDateTime on GleambookUsers(nested.userSince: datetime, nested.alias: string);
+
+create index gbUserSinceIdxInt on GleambookUsers(nested.userSince: int, nested.alias: string);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp
new file mode 100644
index 0000000..9427b77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into GleambookUsers
+([
+{ "id":1, "nested": {"alias":"Bram",    "userSince":datetime("2010-10-16T10:10:00")} },
+{ "id":2, "nested": {"alias":"Donald",  "userSince":"yesterday"} },
+{ "id":3, "nested": {"alias":"Hillery", "userSince":2016} }
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp
new file mode 100644
index 0000000..7c80c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.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 value user
+from GleambookUsers user
+where user.nested.userSince > 900;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp
new file mode 100644
index 0000000..897df02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Index selection for non-enforced nested composite key indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp
new file mode 100644
index 0000000..65b54fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8
+    }
+});
+insert into TestOpen ({
+    "c_id": 9,
+    "nested": {
+        "c_x": 9,
+        "c_d": 3.25
+    }
+});
\ No newline at end of file


[6/6] asterixdb git commit: Merge commit 'b6a70e19' from stabilization-f69489

Posted by mh...@apache.org.
Merge commit 'b6a70e19' from stabilization-f69489

Change-Id: I9f09ed73f27090c5be7b1a665f4591b29a8cca12


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

Branch: refs/heads/master
Commit: f184a1e7b84f6c922b885094281271b4c41a10d6
Parents: 43e0b15 9a61bd2
Author: Murtadha Hubail <mh...@apache.org>
Authored: Sun Dec 9 03:57:02 2018 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Sat Dec 8 18:12:33 2018 -0700

----------------------------------------------------------------------
 .../asterix/app/translator/QueryTranslator.java |   9 +-
 .../01.sqlpp                                    |  40 +++++
 .../02.sqlpp                                    |  40 +++++
 .../03.sqlpp                                    |  46 +++++
 .../04.sqlpp                                    |  46 +++++
 .../05.sqlpp                                    |  46 +++++
 .../06.sqlpp                                    |  46 +++++
 .../07.sqlpp                                    |  46 +++++
 .../non-enforced-composite-key/01.sqlpp         |  38 ++++
 .../non-enforced-composite-key/02.sqlpp         |  38 ++++
 .../non-enforced-composite-key/03.sqlpp         |  40 +++++
 .../non-enforced-composite-key/04.sqlpp         |  39 ++++
 .../non-enforced-composite-key/05.sqlpp         |  39 ++++
 .../non-enforced-composite-key/06.sqlpp         |  39 ++++
 .../non-enforced-composite-key/07.sqlpp         |  39 ++++
 .../non-enforced-composite-key/08.sqlpp         |  39 ++++
 .../non-enforced-composite-key/09.sqlpp         |  39 ++++
 .../non-enforced-composite-key/10.sqlpp         |  39 ++++
 .../non-enforced-composite-key/11.sqlpp         |  39 ++++
 .../non-enforced-composite-key/12.sqlpp         |  42 +++++
 .../01.plan                                     |  22 +++
 .../02.plan                                     |  22 +++
 .../03.plan                                     |  30 ++++
 .../04.plan                                     |  30 ++++
 .../05.plan                                     |  30 ++++
 .../06.plan                                     |  31 ++++
 .../07.plan                                     |  31 ++++
 .../non-enforced-composite-key/01.plan          |  17 ++
 .../non-enforced-composite-key/02.plan          |  15 ++
 .../non-enforced-composite-key/03.plan          |  23 +++
 .../non-enforced-composite-key/04.plan          |  24 +++
 .../non-enforced-composite-key/05.plan          |  24 +++
 .../non-enforced-composite-key/06.plan          |  24 +++
 .../non-enforced-composite-key/07.plan          |  24 +++
 .../non-enforced-composite-key/08.plan          |  24 +++
 .../non-enforced-composite-key/09.plan          |  24 +++
 .../non-enforced-composite-key/10.plan          |  24 +++
 .../non-enforced-composite-key/11.plan          |  25 +++
 .../non-enforced-composite-key/12.plan          |  35 ++++
 ...condary-non-enforced-equi-join-2.1.ddl.sqlpp |  33 ++++
 ...dary-non-enforced-equi-join-2.10.query.sqlpp |  29 +++
 ...ondary-non-enforced-equi-join-2.11.ddl.sqlpp |  20 +++
 ...dary-non-enforced-equi-join-2.2.update.sqlpp | 103 +++++++++++
 ...dary-non-enforced-equi-join-2.3.update.sqlpp | 103 +++++++++++
 ...condary-non-enforced-equi-join-2.4.ddl.sqlpp |  38 ++++
 ...ndary-non-enforced-equi-join-2.5.query.sqlpp |  28 +++
 ...ndary-non-enforced-equi-join-2.6.query.sqlpp |  28 +++
 ...ndary-non-enforced-equi-join-2.7.query.sqlpp |  28 +++
 ...ndary-non-enforced-equi-join-2.8.query.sqlpp |  28 +++
 ...ndary-non-enforced-equi-join-2.9.query.sqlpp |  28 +++
 ...secondary-non-enforced-equi-join.1.ddl.sqlpp |  63 +++++++
 ...ondary-non-enforced-equi-join.2.update.sqlpp | Bin 0 -> 1458 bytes
 ...secondary-non-enforced-equi-join.3.ddl.sqlpp |  22 +++
 ...condary-non-enforced-equi-join.4.query.sqlpp |  25 +++
 ...secondary-non-enforced-equi-join.5.ddl.sqlpp |  20 +++
 .../non-enforced-01/non-enforced-01.1.ddl.sqlpp |  31 ++++
 .../non-enforced-01.10.query.sqlpp              |  25 +++
 .../non-enforced-01.11.query.sqlpp              |  25 +++
 .../non-enforced-01.12.ddl.sqlpp                |  19 ++
 .../non-enforced-01.2.update.sqlpp              | 103 +++++++++++
 .../non-enforced-01/non-enforced-01.3.ddl.sqlpp |  28 +++
 .../non-enforced-01.4.query.sqlpp               |  25 +++
 .../non-enforced-01.5.query.sqlpp               |  25 +++
 .../non-enforced-01.6.query.sqlpp               |  25 +++
 .../non-enforced-01.7.query.sqlpp               |  25 +++
 .../non-enforced-01.8.query.sqlpp               |  25 +++
 .../non-enforced-01.9.query.sqlpp               |  25 +++
 .../non-enforced-02/non-enforced-02.1.ddl.sqlpp |  31 ++++
 .../non-enforced-02.2.update.sqlpp              |  95 ++++++++++
 .../non-enforced-02/non-enforced-02.3.ddl.sqlpp |  28 +++
 .../non-enforced-02.4.query.sqlpp               |  25 +++
 .../non-enforced-02.5.query.sqlpp               |  25 +++
 .../non-enforced-02/non-enforced-02.6.ddl.sqlpp |  20 +++
 .../non-enforced-03/non-enforced-03.1.ddl.sqlpp |  33 ++++
 .../non-enforced-03.2.update.sqlpp              |  27 +++
 .../non-enforced-03.3.query.sqlpp               |  24 +++
 .../non-enforced-03/non-enforced-03.4.ddl.sqlpp |  20 +++
 .../non-enforced-04/non-enforced-04.1.ddl.sqlpp |  31 ++++
 .../non-enforced-04.2.update.sqlpp              | 102 +++++++++++
 .../non-enforced-04/non-enforced-04.3.ddl.sqlpp |  22 +++
 .../non-enforced-04.4.query.sqlpp               |  25 +++
 .../non-enforced-04/non-enforced-04.5.ddl.sqlpp |  20 +++
 ...ee-secondary-non-enforced-equi-join-2.10.adm |   6 +
 ...ree-secondary-non-enforced-equi-join-2.5.adm |  10 ++
 ...ree-secondary-non-enforced-equi-join-2.6.adm |  10 ++
 ...ree-secondary-non-enforced-equi-join-2.7.adm |   8 +
 ...ree-secondary-non-enforced-equi-join-2.8.adm |   8 +
 ...ree-secondary-non-enforced-equi-join-2.9.adm |   9 +
 ...btree-secondary-non-enforced-equi-join.4.adm |   2 +
 .../non-enforced-01/non-enforced-01.10.adm      |   3 +
 .../non-enforced-01/non-enforced-01.11.adm      |   2 +
 .../non-enforced-01/non-enforced-01.4.adm       |   2 +
 .../non-enforced-01/non-enforced-01.5.adm       |   2 +
 .../non-enforced-01/non-enforced-01.6.adm       |   2 +
 .../non-enforced-01/non-enforced-01.7.adm       |   2 +
 .../non-enforced-01/non-enforced-01.8.adm       |   5 +
 .../non-enforced-01/non-enforced-01.9.adm       |   4 +
 .../non-enforced-02/non-enforced-02.4.adm       |   1 +
 .../non-enforced-02/non-enforced-02.5.adm       |   2 +
 .../non-enforced-03/non-enforced-03.3.adm       |   1 +
 .../non-enforced-04/non-enforced-04.4.adm       |   2 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  30 ++++
 .../apache/asterix/metadata/utils/TypeUtil.java | 105 ++++++++---
 hyracks-fullstack/hyracks/hyracks-net/pom.xml   |   6 +
 .../muxdemux/FullFrameChannelReadInterface.java |   4 +-
 .../FullFrameChannelReadInterfaceTest.java      | 180 +++++++++++++++++++
 106 files changed, 3150 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f184a1e7/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f184a1e7/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f184a1e7/hyracks-fullstack/hyracks/hyracks-net/pom.xml
----------------------------------------------------------------------
diff --cc hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 525543f,4ca20ca..2cddf45
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@@ -61,8 -61,10 +61,14 @@@
        <version>${project.version}</version>
      </dependency>
      <dependency>
 +      <groupId>com.fasterxml.jackson.core</groupId>
 +      <artifactId>jackson-databind</artifactId>
 +    </dependency>
++    <dependency>
+       <groupId>org.mockito</groupId>
+       <artifactId>mockito-all</artifactId>
+       <version>2.0.2-beta</version>
+       <scope>test</scope>
+     </dependency>
    </dependencies>
  </project>


[3/6] asterixdb git commit: [NO ISSUE][IDX][COMP] Creating an open index on non-declared fields

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp
new file mode 100644
index 0000000..c04b4d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_i64_on_d on TestOpen(nested.c_d:int64, nested.c_x: int);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp
new file mode 100644
index 0000000..1c07ea6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-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 value t.nested.c_x
+from TestOpen t
+where t.nested.c_d = 3.25
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm
new file mode 100644
index 0000000..c45a808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm
@@ -0,0 +1,6 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 4, "c2": 104 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm
new file mode 100644
index 0000000..8643d5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm
@@ -0,0 +1,10 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm
new file mode 100644
index 0000000..8643d5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm
@@ -0,0 +1,10 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm
new file mode 100644
index 0000000..97b278c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm
@@ -0,0 +1,9 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 103 }
+{ "c1": 2, "c2": 101 }
+{ "c1": 2, "c2": 103 }
+{ "c1": 3, "c2": 101 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm
new file mode 100644
index 0000000..a4b9391
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm
@@ -0,0 +1,2 @@
+{ "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
+{ "aid": 34, "bid": 57, "authors": "" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm
new file mode 100644
index 0000000..04080b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm
@@ -0,0 +1,3 @@
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm
new file mode 100644
index 0000000..429a13d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm
@@ -0,0 +1,2 @@
+{ "res": 4 }
+{ "res": 5 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm
new file mode 100644
index 0000000..f36e389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm
@@ -0,0 +1,2 @@
+{ "res": 3 }
+{ "res": 7 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm
new file mode 100644
index 0000000..9897674
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm
@@ -0,0 +1,2 @@
+{ "res": 1 }
+{ "res": 3 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm
new file mode 100644
index 0000000..c1c06bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm
@@ -0,0 +1,2 @@
+{ "res": 5 }
+{ "res": 7 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm
new file mode 100644
index 0000000..c1c06bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm
@@ -0,0 +1,2 @@
+{ "res": 5 }
+{ "res": 7 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm
new file mode 100644
index 0000000..e6e7ad6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm
@@ -0,0 +1,5 @@
+{ "res": 2 }
+{ "res": 3 }
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm
new file mode 100644
index 0000000..db56195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm
@@ -0,0 +1,4 @@
+{ "res": 3 }
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm
new file mode 100644
index 0000000..66fa150
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm
@@ -0,0 +1 @@
+{ "res": 3 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm
new file mode 100644
index 0000000..18cc982
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm
@@ -0,0 +1,2 @@
+{ "res": 3 }
+{ "res": 5 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm
new file mode 100644
index 0000000..31ca720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm
@@ -0,0 +1 @@
+{ "id": 3, "nested": { "alias": "Hillery", "userSince": 2016 } }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm
new file mode 100644
index 0000000..05d6df6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm
@@ -0,0 +1,2 @@
+4
+9
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/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 f5a2ab4..37eaf79 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4284,6 +4284,16 @@
           <output-dir compare="Text">word-jaccard-inline</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+        </compilation-unit>
+      </test-case>
     </test-group>
     <test-group name="nested-open-index/index-leftouterjoin">
       <test-case FilePath="nested-open-index/index-leftouterjoin">
@@ -4388,6 +4398,26 @@
           <output-dir compare="Text">rtree-secondary-index</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-01">
+          <output-dir compare="Text">non-enforced-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-02">
+          <output-dir compare="Text">non-enforced-02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-03">
+          <output-dir compare="Text">non-enforced-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-04">
+          <output-dir compare="Text">non-enforced-04</output-dir>
+        </compilation-unit>
+      </test-case>
     </test-group>
     <test-group name="nested-open-index/highly-open-highly-nested">
       <test-case FilePath="nested-open-index/highly-open-highly-nested">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index 4ef3933..efe795b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -24,7 +24,10 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -51,8 +54,9 @@ public class TypeUtil {
      */
     public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
             List<Index> indexes) throws AlgebricksException {
-        ARecordType enforcedRecordType = recordType;
+        IAType enforcedRecordType = recordType;
         ARecordType enforcedMetaType = metaType;
+        List<String> subFieldName;
         for (Index index : indexes) {
             if (!index.isSecondaryIndex() || !index.isOverridingKeyFieldTypes()) {
                 continue;
@@ -61,21 +65,28 @@ public class TypeUtil {
                 throw new AlgebricksException("Indexing an open field is only supported on the record part");
             }
             for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                Deque<Pair<ARecordType, String>> nestedTypeStack = new ArrayDeque<>();
+                // keeps track of a record type and a field name in that record type
+                Deque<Pair<IAType, String>> nestedTypeStack = new ArrayDeque<>();
                 List<String> splits = index.getKeyFieldNames().get(i);
-                ARecordType nestedFieldType = enforcedRecordType;
+                IAType nestedFieldType = enforcedRecordType;
                 boolean openRecords = false;
                 String bridgeName = nestedFieldType.getTypeName();
                 int j;
-                // Build the stack for the enforced type
+                // enforcedRecordType must always be/stay as ARecordType
+                validateRecord(enforcedRecordType);
+                // build the stack for the enforced type, stack of a mixture of ARecord and AUnion(ARecord) types
+                // try to build up to the last record field, e.g. for a.b.c.d.e, build up to and including "d"
                 for (j = 1; j < splits.size(); j++) {
                     nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
                     bridgeName = nestedFieldType.getTypeName();
-                    nestedFieldType = (ARecordType) enforcedRecordType.getSubFieldType(splits.subList(0, j));
+                    subFieldName = splits.subList(0, j);
+                    nestedFieldType = ((ARecordType) enforcedRecordType).getSubFieldType(subFieldName);
                     if (nestedFieldType == null) {
                         openRecords = true;
                         break;
                     }
+                    // nestedFieldType (i.e. nested record field) must be either ARecordType or AUnion(ARecordType)
+                    validateNestedRecord(nestedFieldType, subFieldName);
                 }
                 if (openRecords) {
                     // create the smallest record
@@ -87,19 +98,23 @@ public class TypeUtil {
                         enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
                                 new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
                     }
-                    // Bridge the gap
-                    Pair<ARecordType, String> gapPair = nestedTypeStack.pop();
-                    ARecordType parent = gapPair.first;
+                    // bridge the gap. Update the parent type to include the new optional field, e.g. c.d.e
+                    Pair<IAType, String> gapPair = nestedTypeStack.pop();
+                    ARecordType parent = (ARecordType) TypeComputeUtils.getActualType(gapPair.first);
 
+                    // parent type must be "open" to allow inclusion of the non-declared field
                     IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
                             new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
                     enforcedRecordType = new ARecordType(bridgeName,
                             ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
                             parentFieldTypes, true);
+                    // make nullable/missable if the original parent was nullable/missable
+                    enforcedRecordType = keepUnknown(gapPair.first, (ARecordType) enforcedRecordType);
                 } else {
-                    //Schema is closed all the way to the field
-                    //enforced fields are either null or strongly typed
-                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(nestedFieldType);
+                    // schema is closed all the way to the field. Enforced fields are either null or strongly typed
+                    // e.g. nestedFieldType = a.b.c.d
+                    ARecordType lastNestedRecord = (ARecordType) TypeComputeUtils.getActualType(nestedFieldType);
+                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(lastNestedRecord);
                     // if a an enforced field already exists and the type is correct
                     IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
                     if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
@@ -108,44 +123,49 @@ public class TypeUtil {
                     }
                     if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
                             index.getKeyFieldTypes().get(i).getTypeTag())) {
-                        throw new AlgebricksException("Cannot enforce field " + index.getKeyFieldNames().get(i)
-                                + " to have type " + index.getKeyFieldTypes().get(i));
+                        throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                                "Cannot enforce field \"" + String.join(".", index.getKeyFieldNames().get(i))
+                                        + "\" to have type " + index.getKeyFieldTypes().get(i));
                     }
                     if (enforcedFieldType == null) {
                         recordNameTypesMap.put(splits.get(splits.size() - 1),
                                 AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
                     }
-                    enforcedRecordType = new ARecordType(nestedFieldType.getTypeName(),
+                    enforcedRecordType = new ARecordType(lastNestedRecord.getTypeName(),
                             recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
                             recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
-                            nestedFieldType.isOpen());
+                            lastNestedRecord.isOpen());
+                    // make nullable/missable if the original nestedFieldType was nullable/missable
+                    enforcedRecordType = keepUnknown(nestedFieldType, (ARecordType) enforcedRecordType);
                 }
 
                 // Create the enforced type for the nested fields in the schema, from the ground up
                 if (!nestedTypeStack.isEmpty()) {
                     while (!nestedTypeStack.isEmpty()) {
-                        Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
-                        ARecordType nestedRecType = nestedTypePair.first;
+                        Pair<IAType, String> nestedType = nestedTypeStack.pop();
+                        ARecordType nestedRecType = (ARecordType) TypeComputeUtils.getActualType(nestedType.first);
                         IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] =
-                                enforcedRecordType;
+                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedType.second)] = enforcedRecordType;
                         enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
                                 nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
+                        // make nullable/missable if the original nestedRecType was nullable/missable
+                        enforcedRecordType = keepUnknown(nestedType.first, (ARecordType) enforcedRecordType);
                     }
                 }
             }
         }
-        return new Pair<>(enforcedRecordType, enforcedMetaType);
+        // the final enforcedRecordType built must be ARecordType since the original dataset rec. type can't be nullable
+        validateRecord(enforcedRecordType);
+        return new Pair<>((ARecordType) enforcedRecordType, enforcedMetaType);
     }
 
     /**
      * Creates a map from name to type for fields in the passed type
      *
-     * @param recordType
-     *            the type to be mapped
+     * @param recordType the type to be mapped
      * @return a map mapping all fields to their types
      */
-    public static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+    private static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
         LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
         for (int j = 0; j < recordType.getFieldNames().length; j++) {
             recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
@@ -153,4 +173,43 @@ public class TypeUtil {
         return recordNameTypesMap;
     }
 
+    /**
+     * Maintains the {@code updatedRecordType} as nullable/missable (?) in case it was originally nullable/missable
+     * @param originalRecordType the original record type
+     * @param updatedRecordType the original record type being enforced/modified with new non-declared fields included
+     * @return {@code updatedRecordType}
+     */
+    private static IAType keepUnknown(IAType originalRecordType, ARecordType updatedRecordType) {
+        if (originalRecordType.getTypeTag() == ATypeTag.UNION) {
+            return AUnionType.createUnknownableType(updatedRecordType, updatedRecordType.getTypeName());
+        }
+        return updatedRecordType;
+    }
+
+    /**
+     * Makes sure the dataset record type being enforced/modified stays as a pure record type
+     * @param enforcedDatasetRecordType the dataset record type enforced and modified by adding the extra fields indexed
+     */
+    private static void validateRecord(IAType enforcedDatasetRecordType) {
+        if (enforcedDatasetRecordType.getTypeTag() != ATypeTag.OBJECT) {
+            throw new IllegalStateException("The dataset type must be a record type to be able to build an index");
+        }
+    }
+
+    /**
+     * Makes sure the chain of fields accessed and leading to the indexed field are all valid record types.
+     * E.g. for CREATE INDEX idx on ds(a.b.c.d: int) validate that a, b and c are all valid record types (?).
+     * @param nestedRecordType the nested record field being accessed
+     * @param fieldName the name of the nested record field
+     * @throws AsterixException when supplying bad fields, e.g. CREATE INDEX i on ds(a.b: int, a.b.c: int) (mostly
+     * for non-declared fields)
+     */
+    private static void validateNestedRecord(IAType nestedRecordType, List<String> fieldName) throws AsterixException {
+        IAType actualType = TypeComputeUtils.getActualType(nestedRecordType);
+        if (actualType.getTypeTag() != ATypeTag.OBJECT) {
+            String fName = String.join(".", fieldName);
+            throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                    "Field accessor is not defined for \"" + fName + "\" of type " + actualType.getTypeTag());
+        }
+    }
 }


[2/6] asterixdb git commit: [NO ISSUE][OTH] Pass Request Parameters To Extensions

Posted by mh...@apache.org.
[NO ISSUE][OTH] Pass Request Parameters To Extensions

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

Details:
- Pass request parameters to create/drop dataverse
  statements to allow extensions to use optional
  request parameters.

Change-Id: I284560ea3a7e3d1b36d73a8cfcdc85bf7b5a7f4e
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3044
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: d79363249d8d7106b3bbe352ead18eb01fe9b973
Parents: 821c072
Author: Murtadha Hubail <mh...@apache.org>
Authored: Wed Nov 28 17:07:46 2018 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Fri Nov 30 14:39:38 2018 -0800

----------------------------------------------------------------------
 .../org/apache/asterix/app/translator/QueryTranslator.java  | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d7936324/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 8e86b9c..f619ab6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -309,7 +309,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                         activeDataverse = handleUseDataverseStatement(metadataProvider, stmt);
                         break;
                     case CREATE_DATAVERSE:
-                        handleCreateDataverseStatement(metadataProvider, stmt);
+                        handleCreateDataverseStatement(metadataProvider, stmt, requestParameters);
                         break;
                     case DATASET_DECL:
                         handleCreateDatasetStatement(metadataProvider, stmt, hcc, requestParameters);
@@ -324,7 +324,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                         handleCreateNodeGroupStatement(metadataProvider, stmt);
                         break;
                     case DATAVERSE_DROP:
-                        handleDataverseDropStatement(metadataProvider, stmt, hcc);
+                        handleDataverseDropStatement(metadataProvider, stmt, hcc, requestParameters);
                         break;
                     case DATASET_DROP:
                         handleDatasetDropStatement(metadataProvider, stmt, hcc, requestParameters);
@@ -466,7 +466,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
-    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
+            IRequestParameters requestParameters) throws Exception {
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         String dvName = stmtCreateDataverse.getDataverseName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1227,7 +1228,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     }
 
     protected void handleDataverseDropStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
+            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
         SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String dataverseName = stmtDelete.getDataverseName().getValue();


[5/6] asterixdb git commit: [NO ISSUE][IDX][COMP] Creating an open index on non-declared fields

Posted by mh...@apache.org.
[NO ISSUE][IDX][COMP] Creating an open index on non-declared fields

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

Details:
This patch is to fix creating an open index on non-declared fields.
When compiling the CREATE INDEX DDL statement, it is assumed
that all nested record fields will be of type ARecordType. However, for
open indexes, any nested non-declared record field will be optional.
In this case, the type will be AUnion(ARecordType) which results
in type casting exceptions for certain situations. An example is:
CREATE INDEX idx ON ds(a.b.c: int, a.b.d: string). Both a and b could
have the type AUnion(ARecordType). This patch takes care of such cases.

Change-Id: I9f09ed73f27090c5be7b1a665f4591b29a8cda12
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3045
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: 9a61bd2ce4f54f535c44ca247ecf035cb6a70e19
Parents: d793632
Author: Ali Alsuliman <al...@gmail.com>
Authored: Thu Nov 29 22:29:42 2018 -0800
Committer: Ali Alsuliman <al...@gmail.com>
Committed: Tue Dec 4 00:01:47 2018 -0800

----------------------------------------------------------------------
 .../01.sqlpp                                    |  40 +++++++
 .../02.sqlpp                                    |  40 +++++++
 .../03.sqlpp                                    |  46 ++++++++
 .../04.sqlpp                                    |  46 ++++++++
 .../05.sqlpp                                    |  46 ++++++++
 .../06.sqlpp                                    |  46 ++++++++
 .../07.sqlpp                                    |  46 ++++++++
 .../non-enforced-composite-key/01.sqlpp         |  38 +++++++
 .../non-enforced-composite-key/02.sqlpp         |  38 +++++++
 .../non-enforced-composite-key/03.sqlpp         |  40 +++++++
 .../non-enforced-composite-key/04.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/05.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/06.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/07.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/08.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/09.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/10.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/11.sqlpp         |  39 +++++++
 .../non-enforced-composite-key/12.sqlpp         |  42 ++++++++
 .../01.plan                                     |  22 ++++
 .../02.plan                                     |  22 ++++
 .../03.plan                                     |  30 ++++++
 .../04.plan                                     |  30 ++++++
 .../05.plan                                     |  30 ++++++
 .../06.plan                                     |  31 ++++++
 .../07.plan                                     |  31 ++++++
 .../non-enforced-composite-key/01.plan          |  17 +++
 .../non-enforced-composite-key/02.plan          |  15 +++
 .../non-enforced-composite-key/03.plan          |  23 ++++
 .../non-enforced-composite-key/04.plan          |  24 +++++
 .../non-enforced-composite-key/05.plan          |  24 +++++
 .../non-enforced-composite-key/06.plan          |  24 +++++
 .../non-enforced-composite-key/07.plan          |  24 +++++
 .../non-enforced-composite-key/08.plan          |  24 +++++
 .../non-enforced-composite-key/09.plan          |  24 +++++
 .../non-enforced-composite-key/10.plan          |  24 +++++
 .../non-enforced-composite-key/11.plan          |  25 +++++
 .../non-enforced-composite-key/12.plan          |  35 +++++++
 ...condary-non-enforced-equi-join-2.1.ddl.sqlpp |  33 ++++++
 ...dary-non-enforced-equi-join-2.10.query.sqlpp |  29 +++++
 ...ondary-non-enforced-equi-join-2.11.ddl.sqlpp |  20 ++++
 ...dary-non-enforced-equi-join-2.2.update.sqlpp | 103 ++++++++++++++++++
 ...dary-non-enforced-equi-join-2.3.update.sqlpp | 103 ++++++++++++++++++
 ...condary-non-enforced-equi-join-2.4.ddl.sqlpp |  38 +++++++
 ...ndary-non-enforced-equi-join-2.5.query.sqlpp |  28 +++++
 ...ndary-non-enforced-equi-join-2.6.query.sqlpp |  28 +++++
 ...ndary-non-enforced-equi-join-2.7.query.sqlpp |  28 +++++
 ...ndary-non-enforced-equi-join-2.8.query.sqlpp |  28 +++++
 ...ndary-non-enforced-equi-join-2.9.query.sqlpp |  28 +++++
 ...secondary-non-enforced-equi-join.1.ddl.sqlpp |  63 +++++++++++
 ...ondary-non-enforced-equi-join.2.update.sqlpp | Bin 0 -> 1458 bytes
 ...secondary-non-enforced-equi-join.3.ddl.sqlpp |  22 ++++
 ...condary-non-enforced-equi-join.4.query.sqlpp |  25 +++++
 ...secondary-non-enforced-equi-join.5.ddl.sqlpp |  20 ++++
 .../non-enforced-01/non-enforced-01.1.ddl.sqlpp |  31 ++++++
 .../non-enforced-01.10.query.sqlpp              |  25 +++++
 .../non-enforced-01.11.query.sqlpp              |  25 +++++
 .../non-enforced-01.12.ddl.sqlpp                |  19 ++++
 .../non-enforced-01.2.update.sqlpp              | 103 ++++++++++++++++++
 .../non-enforced-01/non-enforced-01.3.ddl.sqlpp |  28 +++++
 .../non-enforced-01.4.query.sqlpp               |  25 +++++
 .../non-enforced-01.5.query.sqlpp               |  25 +++++
 .../non-enforced-01.6.query.sqlpp               |  25 +++++
 .../non-enforced-01.7.query.sqlpp               |  25 +++++
 .../non-enforced-01.8.query.sqlpp               |  25 +++++
 .../non-enforced-01.9.query.sqlpp               |  25 +++++
 .../non-enforced-02/non-enforced-02.1.ddl.sqlpp |  31 ++++++
 .../non-enforced-02.2.update.sqlpp              |  95 +++++++++++++++++
 .../non-enforced-02/non-enforced-02.3.ddl.sqlpp |  28 +++++
 .../non-enforced-02.4.query.sqlpp               |  25 +++++
 .../non-enforced-02.5.query.sqlpp               |  25 +++++
 .../non-enforced-02/non-enforced-02.6.ddl.sqlpp |  20 ++++
 .../non-enforced-03/non-enforced-03.1.ddl.sqlpp |  33 ++++++
 .../non-enforced-03.2.update.sqlpp              |  27 +++++
 .../non-enforced-03.3.query.sqlpp               |  24 +++++
 .../non-enforced-03/non-enforced-03.4.ddl.sqlpp |  20 ++++
 .../non-enforced-04/non-enforced-04.1.ddl.sqlpp |  31 ++++++
 .../non-enforced-04.2.update.sqlpp              | 102 ++++++++++++++++++
 .../non-enforced-04/non-enforced-04.3.ddl.sqlpp |  22 ++++
 .../non-enforced-04.4.query.sqlpp               |  25 +++++
 .../non-enforced-04/non-enforced-04.5.ddl.sqlpp |  20 ++++
 ...ee-secondary-non-enforced-equi-join-2.10.adm |   6 ++
 ...ree-secondary-non-enforced-equi-join-2.5.adm |  10 ++
 ...ree-secondary-non-enforced-equi-join-2.6.adm |  10 ++
 ...ree-secondary-non-enforced-equi-join-2.7.adm |   8 ++
 ...ree-secondary-non-enforced-equi-join-2.8.adm |   8 ++
 ...ree-secondary-non-enforced-equi-join-2.9.adm |   9 ++
 ...btree-secondary-non-enforced-equi-join.4.adm |   2 +
 .../non-enforced-01/non-enforced-01.10.adm      |   3 +
 .../non-enforced-01/non-enforced-01.11.adm      |   2 +
 .../non-enforced-01/non-enforced-01.4.adm       |   2 +
 .../non-enforced-01/non-enforced-01.5.adm       |   2 +
 .../non-enforced-01/non-enforced-01.6.adm       |   2 +
 .../non-enforced-01/non-enforced-01.7.adm       |   2 +
 .../non-enforced-01/non-enforced-01.8.adm       |   5 +
 .../non-enforced-01/non-enforced-01.9.adm       |   4 +
 .../non-enforced-02/non-enforced-02.4.adm       |   1 +
 .../non-enforced-02/non-enforced-02.5.adm       |   2 +
 .../non-enforced-03/non-enforced-03.3.adm       |   1 +
 .../non-enforced-04/non-enforced-04.4.adm       |   2 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  30 ++++++
 .../apache/asterix/metadata/utils/TypeUtil.java | 105 +++++++++++++++----
 102 files changed, 2957 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
new file mode 100644
index 0000000..792dde1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.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  : Testing that creating a nested composite key open index is successful and being used.
+ * Expected Res : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.openType as {
+  id : int
+};
+
+create  dataset ds1(openType) primary key id;
+create  dataset ds2(openType) primary key id;
+
+create  index idx  on ds2 (nested.fname:string, nested.lname:string);
+
+select element {'a':a,'b':b}
+from ds1 as a, ds2 as b
+where to_string(a.nested.fname) /*+ indexnl */ = b.nested.fname;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
new file mode 100644
index 0000000..438ea9b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.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  : Testing that creating a nested composite key open index is successful and being used.
+ * Expected Res : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.openType as {
+  id : int
+};
+
+create  dataset ds1(openType) primary key id;
+create  dataset ds2(openType) primary key id;
+
+create  index idx  on ds2 (nested.fname:string, nested.address.street:string, nested.address.zip:string);
+
+select element {'a':a,'b':b}
+from ds1 as a, ds2 as b
+where to_string(a.nested.fname) /*+ indexnl */ = b.nested.fname;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp
new file mode 100644
index 0000000..043fdf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.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  : No index join because there's no hint and the probe type is unknown, non-enforced nested composite key
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp
new file mode 100644
index 0000000..8e71939
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.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  : No index join because the probe type is unknown, non-enforced nested composite key
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp
new file mode 100644
index 0000000..49d6a46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.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  : No index join because there's no hint, non-enforced nested composite key index
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
new file mode 100644
index 0000000..b10915d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.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  : Index join because there's a hint and the probe type is known (string), non-enforced nested composite
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
new file mode 100644
index 0000000..9dce697
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.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  : Index join because there's a hint and the probe type is known (bigint), non-enforced nested composite
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.nested.c_i64) /*+ indexnl */ = t2.nested.c_i64
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
new file mode 100644
index 0000000..6f97c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
@@ -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     : Testing that creating a nested composite key open index is successful and being used.
+ *  Expected Result : Success
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.TestType as {
+  id: int
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (nested.fname:string, nested.lname:string);
+
+select element emp
+from  testdst as emp
+where emp.nested.fname > 'Roger';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
new file mode 100644
index 0000000..0fc5b4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
@@ -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     : Testing that creating a nested composite key open index is successful and being used.
+ *  Expected Result : Success
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.TestType as {
+  id: int
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (nested.address.zip:string, nested.fname:string, nested.address.street:string);
+
+select element emp
+from  testdst as emp
+where emp.nested.address.zip > '97777';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
new file mode 100644
index 0000000..45f87fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.x: string);
+
+select t.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
new file mode 100644
index 0000000..534df4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_s on TestOpen(nested.c_s:string, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
new file mode 100644
index 0000000..ae384bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
new file mode 100644
index 0000000..d7aa320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
new file mode 100644
index 0000000..bc7b0a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2.0
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
new file mode 100644
index 0000000..094c8b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
new file mode 100644
index 0000000..37a6e97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2.5
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
new file mode 100644
index 0000000..6cd28df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_d on TestOpen(nested.c_d:double, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_d >= 3.25
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
new file mode 100644
index 0000000..9024f4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 499 and t.nested.c_i8 < 99999
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
new file mode 100644
index 0000000..546a041
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.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     : Testing that creating a nested composite key open index is successful and being used
+  *                    when multiple indexes match.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_3 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+create index idx_4 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2 and t.nested.c_i64 < 3
+order by t.nested.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
new file mode 100644
index 0000000..8d3d87e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |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|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
new file mode 100644
index 0000000..1cc11dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |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|
+                    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
new file mode 100644
index 0000000..ce57c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |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|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
new file mode 100644
index 0000000..ce57c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |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|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
new file mode 100644
index 0000000..a90431b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |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|
+                    -- HASH_PARTITION_EXCHANGE [$$40]  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
new file mode 100644
index 0000000..af9bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
new file mode 100644
index 0000000..af9bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
new file mode 100644
index 0000000..fc81846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$19(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
new file mode 100644
index 0000000..b85b5c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$22(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
new file mode 100644
index 0000000..e8619e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$19(ASC)]  |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|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$26(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
new file mode 100644
index 0000000..a2fcba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
new file mode 100644
index 0000000..a2fcba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(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|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9a61bd2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(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|
\ No newline at end of file