You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by vj...@apache.org on 2023/11/10 03:13:12 UTC

(phoenix) branch master updated: PHOENIX-7067 View indexes should be created only on non overlapping updatable views (#1709)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 62ccb3b815 PHOENIX-7067 View indexes should be created only on non overlapping updatable views (#1709)
62ccb3b815 is described below

commit 62ccb3b815aca1c1cd2f2b9fb8feab024bec834f
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Thu Nov 9 18:13:07 2023 -0900

    PHOENIX-7067 View indexes should be created only on non overlapping updatable views (#1709)
---
 .../org/apache/phoenix/end2end/AlterTableIT.java   |   75 +-
 .../phoenix/end2end/MetaDataEndpointImplIT.java    |   26 +-
 .../end2end/ViewExtendsPkRestrictionsIT.java       | 1169 ++++++++++++++++++++
 .../java/org/apache/phoenix/end2end/ViewTTLIT.java |   65 +-
 .../phoenix/compile/CreateTableCompiler.java       |   61 +
 .../apache/phoenix/exception/SQLExceptionCode.java |    4 +
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java      |    2 +
 .../query/DelegateConnectionQueryServices.java     |    2 +-
 .../phoenix/query/DelegateQueryServices.java       |    4 +-
 .../org/apache/phoenix/query/QueryServices.java    |   11 +
 .../org/apache/phoenix/schema/MetaDataClient.java  |   74 +-
 11 files changed, 1420 insertions(+), 73 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index b6c1cc1b12..8908af2047 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -1182,50 +1182,93 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testDeclaringColumnAsRowTimestamp() throws Exception {
+    public void testIndexColumnAsRowTimestamp() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
-            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class); 
-            PTable table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName));
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName
+                    + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR "
+                    + "CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable table =
+                    phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName));
             // Assert that the column shows up as row time stamp in the cache.
             assertTrue(table.getColumnForColumnName("PK1").isRowTimestamp());
             assertFalse(table.getColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName, "PK1");
-            
+
             String dataTableName2 = BaseTest.generateUniqueName();
             String dataTableFullName2 = SchemaUtil.getTableName(schemaName, dataTableName2);
-            conn.createStatement().execute("CREATE IMMUTABLE TABLE " + dataTableFullName2 + " (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
+            conn.createStatement().execute("CREATE IMMUTABLE TABLE " + dataTableFullName2 +
+                    " (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
             table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName2));
             // Assert that the column shows up as row time stamp in the cache.
             assertFalse(table.getColumnForColumnName("PK1").isRowTimestamp());
             assertTrue(table.getColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName2, "PK2");
-            
+
             // Create an index on a table has a row time stamp pk column. The column should show up as a row time stamp column for the index too. 
-            conn.createStatement().execute("CREATE INDEX " + indexTableName + "  ON " + dataTableFullName2 + " (KV1) include (KV2)");
-            PTable indexTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), indexTableFullName));
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexTableName + "  ON " + dataTableFullName2 +
+                            " (KV1) include (KV2)");
+            PTable indexTable =
+                    phxConn.getTable(new PTableKey(phxConn.getTenantId(), indexTableFullName));
             String indexColName = IndexUtil.getIndexColumnName(table.getColumnForColumnName("PK2"));
             // Assert that the column shows up as row time stamp in the cache.
             assertTrue(indexTable.getColumnForColumnName(indexColName).isRowTimestamp());
             assertIsRowTimestampSet(schemaName, indexTableName, indexColName);
+        }
+    }
+
+    @Test
+    public void testDeclaringColumnAsRowTimestamp() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName
+                    + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR "
+                    + "CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable table =
+                    phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName));
+            // Assert that the column shows up as row time stamp in the cache.
+            assertTrue(table.getColumnForColumnName("PK1").isRowTimestamp());
+            assertFalse(table.getColumnForColumnName("PK2").isRowTimestamp());
+            assertIsRowTimestampSet(schemaName, dataTableName, "PK1");
+
+            String dataTableName2 = BaseTest.generateUniqueName();
+            String dataTableFullName2 = SchemaUtil.getTableName(schemaName, dataTableName2);
+            conn.createStatement().execute("CREATE IMMUTABLE TABLE " + dataTableFullName2 +
+                    " (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
+            table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName2));
+            // Assert that the column shows up as row time stamp in the cache.
+            assertFalse(table.getColumnForColumnName("PK1").isRowTimestamp());
+            assertTrue(table.getColumnForColumnName("PK2").isRowTimestamp());
+            assertIsRowTimestampSet(schemaName, dataTableName2, "PK2");
+
             String viewTableName2 = dataTableName2 + "_VIEW";
             String viewTableFullName2 = SchemaUtil.getTableName(schemaName, viewTableName2);
             // Creating a view with a row_timestamp column in its pk constraint is not allowed
             try {
-                conn.createStatement().execute("CREATE VIEW " + viewTableFullName2 + " (KV3 VARCHAR, KV4 DATE, KV5 INTEGER, CONSTRAINT PK PRIMARY KEY (KV3, KV4 ROW_TIMESTAMP) ) AS SELECT * FROM " + dataTableFullName2);
-                fail("Creating a view with a row_timestamp column in its pk constraint is not allowed");
+                conn.createStatement().execute("CREATE VIEW " + viewTableFullName2
+                        + " (KV3 VARCHAR, KV4 DATE, KV5 INTEGER, "
+                        + "CONSTRAINT PK PRIMARY KEY (KV3, KV4 ROW_TIMESTAMP) ) AS SELECT * FROM "
+                        + dataTableFullName2);
+                fail("Creating a view with a row_timestamp column in its pk constraint is not "
+                        + "allowed");
             } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.ROWTIMESTAMP_NOT_ALLOWED_ON_VIEW.getErrorCode(), e.getErrorCode());
+                assertEquals(SQLExceptionCode.ROWTIMESTAMP_NOT_ALLOWED_ON_VIEW.getErrorCode(),
+                        e.getErrorCode());
             }
-            
+
             // Make sure that the base table column declared as row_timestamp is also row_timestamp for view
-            conn.createStatement().execute("CREATE VIEW " + viewTableFullName2 + " (KV3 VARCHAR, KV4 VARCHAR, KV5 INTEGER, CONSTRAINT PK PRIMARY KEY (KV3, KV4) ) AS SELECT * FROM " + dataTableFullName2);
-            PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), viewTableFullName2));
+            conn.createStatement().execute("CREATE VIEW " + viewTableFullName2
+                    + " (KV3 VARCHAR, KV4 VARCHAR, KV5 INTEGER, "
+                    + "CONSTRAINT PK PRIMARY KEY (KV3, KV4)) AS SELECT * FROM "
+                    + dataTableFullName2);
+            PTable view =
+                    phxConn.getTable(new PTableKey(phxConn.getTenantId(), viewTableFullName2));
             assertNotNull(view.getPKColumn("PK2"));
             assertTrue(view.getPKColumn("PK2").isRowTimestamp());
         }
     }
-    
+
     private void assertIsRowTimestampSet(String schemaName, String tableName, String columnName) throws SQLException {
         String sql = "SELECT IS_ROW_TIMESTAMP FROM \"SYSTEM\".\"CATALOG\" WHERE "
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ tableName + "') AND\n"
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
index e30a6e0236..7f0c9d9e25 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
@@ -101,7 +101,7 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testUpsertIntoChildViewWithPKAndIndex() throws Exception {
+    public void testUpsertIntoChildViewWithPK() throws Exception {
         String baseTable = generateUniqueName();
         String view = generateUniqueName();
         String childView = generateUniqueName();
@@ -117,12 +117,7 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
                     + "V4 VARCHAR CONSTRAINT PKVIEW PRIMARY KEY(V2, V3)) AS SELECT * FROM " 
                     + baseTable + " WHERE KEY_PREFIX = '0CY'";
             conn.createStatement().execute(view1DDL);
-    
-            // Create an Index on the base view
-            String view1Index = generateUniqueName() + "_IDX";
-            conn.createStatement().execute("CREATE INDEX " + view1Index + 
-                " ON " + view + " (V2, V3) include (V1, V4)");
-    
+
             // Create a child view with primary key constraint
             String childViewDDL = "CREATE VIEW IF NOT EXISTS " + childView 
                     + " (V5 VARCHAR NOT NULL, V6 VARCHAR NOT NULL CONSTRAINT PK PRIMARY KEY "
@@ -137,7 +132,7 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testUpsertIntoTenantChildViewWithPKAndIndex() throws Exception {
+    public void testUpsertIntoTenantChildViewWithPK() throws Exception {
         String baseTable = generateUniqueName();
         String view = generateUniqueName();
         String childView = generateUniqueName();
@@ -155,11 +150,6 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
                     + baseTable + " WHERE KEY_PREFIX = '0CY'";
             conn.createStatement().execute(view1DDL);
     
-            // Create an Index on the base view
-            String view1Index = generateUniqueName() + "_IDX";
-            conn.createStatement().execute("CREATE INDEX " + view1Index + 
-                " ON " + view + " (V2, V3) include (V1, V4)");
-    
             // Create a child view with primary key constraint owned by tenant
             Properties tenantProps = new Properties();
             tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
@@ -167,12 +157,12 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
                 String childViewDDL = "CREATE VIEW IF NOT EXISTS " + childView 
                         + " (V5 VARCHAR NOT NULL, V6 VARCHAR NOT NULL CONSTRAINT PK PRIMARY KEY "
                         + "(V5, V6)) AS SELECT * FROM " + view;
-                conn.createStatement().execute(childViewDDL);
+                tenantConn.createStatement().execute(childViewDDL);
+                String upsert = "UPSERT INTO " + childView + " (TENANT_ID, V2, V3, V5, V6) "
+                        + "VALUES ('00D005000000000',  'zzzzz', 10, 'zzzzz', 'zzzzz')";
+                tenantConn.createStatement().executeUpdate(upsert);
+                tenantConn.commit();
             }
-            
-            String upsert = "UPSERT INTO " + childView + " (TENANT_ID, V2, V3, V5, V6) "
-                    + "VALUES ('00D005000000000',  'zzzzz', 10, 'zzzzz', 'zzzzz')";
-            conn.createStatement().executeUpdate(upsert);
             conn.commit();
         }
     }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewExtendsPkRestrictionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewExtendsPkRestrictionsIT.java
new file mode 100644
index 0000000000..05071ecba7
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewExtendsPkRestrictionsIT.java
@@ -0,0 +1,1169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.phoenix.end2end;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK;
+import static org.apache.phoenix.exception.SQLExceptionCode.VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES;
+import static org.apache.phoenix.query.QueryServices.DISABLE_VIEW_SUBTREE_VALIDATION;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for restrictions associated with view extending primary key of its parent.
+ */
+@Category(ParallelStatsDisabledTest.class)
+public class ViewExtendsPkRestrictionsIT extends ParallelStatsDisabledIT {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ViewExtendsPkRestrictionsIT.class);
+
+    private static final String TENANT_ID = "tenant_01";
+
+    private Connection getTenantConnection(final String tenantId) throws Exception {
+        Properties tenantProps = new Properties();
+        tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        return DriverManager.getConnection(getUrl(), tenantProps);
+    }
+
+    private Connection getTenantConnection(final String tenantId,
+        final boolean disableCreateIndexCheck) throws Exception {
+        Properties tenantProps = new Properties();
+        tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        tenantProps.setProperty(DISABLE_VIEW_SUBTREE_VALIDATION,
+            Boolean.toString(disableCreateIndexCheck));
+        return DriverManager.getConnection(getUrl(), tenantProps);
+    }
+
+    @Test
+    public void testViewExtendsPkWithParentTableIndex1() {
+        final String tableName = generateUniqueName();
+        final String indexName = "idx_" + tableName;
+        final String view01 = "v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute(
+                "CREATE INDEX " + indexName + " ON " + tableName + " (COL3) INCLUDE " + "(COL4)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL PRIMARY KEY, COL5 VARCHAR) AS SELECT * FROM "
+                + tableName + " WHERE COL1 = 'col1'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testSchemaViewExtendsPkWithParentTableIndex1() {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String indexName = "idx_" + tableName;
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (COL3) INCLUDE "
+                + "(COL4)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL PRIMARY KEY, COL5 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE COL1 = 'col1'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithParentTableIndex1() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String indexName = "idx_" + tableName;
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+            stmt.execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (COL3) INCLUDE "
+                + "(COL4)");
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL PRIMARY KEY, COL5 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE COL1 = 'col1'");
+                fail();
+            } catch (SQLException e) {
+                try {
+                    assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                        e.getErrorCode());
+                    assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                        e.getSQLState());
+                    assertTrue(e.getMessage()
+                        .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+                } catch (AssertionError ae) {
+                    LOGGER.error("Exception: ", e);
+                    throw ae;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testViewExtendsPkWithParentTableIndex2() {
+        final String tableName = generateUniqueName();
+        final String indexName = "idx_" + tableName;
+        final String view01 = "v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute(
+                "CREATE INDEX " + indexName + " ON " + tableName + " (COL3) INCLUDE " + "(COL4)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testViewExtendsPkWithViewIndex1() throws Exception {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR)" + " AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+        }
+    }
+
+    @Test
+    public void testViewExtendsPkWithViewIndex2() {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                + "PRIMARY KEY(VCOL2)) AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testSchemaViewExtendsPkWithViewIndex2() {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                + "PRIMARY KEY(VCOL2)) AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex2() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute(
+                    "CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                        + "(COL1, COL2, COL3)");
+                allStmtExecuted = true;
+                tenantStmt.execute(
+                    "CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                        + "PRIMARY KEY(VCOL2)) AS SELECT * FROM " + view01
+                        + " WHERE VCOL1 = 'vcol1'");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testViewExtendsPkWithViewIndex3() {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String view03 = "v03_" + tableName;
+        final String view04 = "v04_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute(
+                "CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col2'");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            allStmtExecuted = true;
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex3() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                stmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                    + "(COL1, COL2, COL3)");
+                allStmtExecuted = true;
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL2 CHAR(10), " + "COL6 VARCHAR PRIMARY KEY) AS "
+                        + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex4() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String schemaName6 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String view05 = SchemaUtil.getTableName(schemaName6, "v05_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                stmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL4 CHAR(10), " + "COL8 VARCHAR) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                    + "(COL1, COL2, COL3)");
+                allStmtExecuted = true;
+                tenantStmt.execute("CREATE VIEW " + view05 + " (VCOL5 CHAR(10), "
+                    + "COL9 VARCHAR, COL10 INTEGER CONSTRAINT pk PRIMARY KEY(VCOL5)) AS "
+                    + "SELECT * FROM " + view04 + " WHERE VCOL4 = 'vcol4'");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex5() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String schemaName6 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String view05 = SchemaUtil.getTableName(schemaName6, "v05_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                stmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL4 CHAR(10), " + "COL8 VARCHAR) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                    + "(COL1, COL2, COL3)");
+                tenantStmt.execute("CREATE VIEW " + view05 + " (VCOL5 CHAR(10), "
+                    + "COL9 VARCHAR, COL10 INTEGER) AS " + "SELECT * FROM " + view04
+                    + " WHERE VCOL4 = 'vcol4'");
+            }
+        }
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex6() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String schemaName6 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String view05 = SchemaUtil.getTableName(schemaName6, "v05_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL4 CHAR(10), " + "COL8 VARCHAR) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                tenantStmt.execute(
+                    "CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                        + "(COL1, COL2, COL3)");
+                allStmtExecuted = true;
+                tenantStmt.execute("CREATE VIEW " + view05 + " (VCOL5 CHAR(10), "
+                    + "COL9 VARCHAR, COL10 INTEGER CONSTRAINT pk PRIMARY KEY(VCOL5)) AS "
+                    + "SELECT * FROM " + view04 + " WHERE VCOL4 = 'vcol4'");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex7() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String schemaName6 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String view05 = SchemaUtil.getTableName(schemaName6, "v05_" + tableName);
+        final String index_table = "idx_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL4 CHAR(10), " + "COL8 VARCHAR) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                allStmtExecuted = true;
+                stmt.execute(
+                    "CREATE INDEX " + index_table + " ON " + fullTableName + " (COL4) INCLUDE "
+                        + "(COL2)");
+                tenantStmt.execute("CREATE VIEW " + view05 + " (VCOL5 CHAR(10), "
+                    + "COL9 VARCHAR, COL10 INTEGER CONSTRAINT pk PRIMARY KEY(VCOL5)) AS "
+                    + "SELECT * FROM " + view04 + " WHERE VCOL4 = 'vcol4'");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewExtendsPkWithViewIndex8() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_table = "idx_" + tableName;
+
+        Properties props = new Properties();
+        props.setProperty(DISABLE_VIEW_SUBTREE_VALIDATION, "true");
+
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID, true)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(8), COL6 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col2'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL4 CHAR(10), " + "COL8 VARCHAR) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                stmt.execute(
+                    "CREATE INDEX " + index_table + " ON " + fullTableName + " (COL4) INCLUDE "
+                        + "(COL2)");
+            }
+        }
+    }
+
+    @Test
+    public void testViewIndexWithChildViewExtendedPk1() throws Exception {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String view03 = "v03_" + tableName;
+        final String view04 = "v04_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+
+        Properties props = new Properties();
+        props.setProperty(DISABLE_VIEW_SUBTREE_VALIDATION, "true");
+
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10) PRIMARY KEY, COL6 VARCHAR)"
+                + " AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+        }
+    }
+
+    @Test
+    public void testViewIndexWithChildViewExtendedPk4() {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String view03 = "v03_" + tableName;
+        final String view04 = "v04_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10) PRIMARY KEY, COL6 VARCHAR)"
+                + " AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            allStmtExecuted = true;
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testSchemaViewIndexWithChildViewExtendedPk1() {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10) PRIMARY KEY, COL6 VARCHAR)"
+                + " AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            allStmtExecuted = true;
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewIndexWithChildViewExtendedPk1() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                tenantStmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view02 + " (VCOL2 CHAR(10) PRIMARY KEY, COL6 VARCHAR)"
+                        + " AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+                tenantStmt.execute(
+                    "CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                        + "(COL1, COL2, COL3)");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                        + "SELECT * FROM " + view03 + " WHERE VCOL1 = 'vcol4'");
+                allStmtExecuted = true;
+                tenantStmt.execute(
+                    "CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                        + "(COL1, COL2, COL3)");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testViewIndexWithChildViewExtendedPk2() {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String view03 = "v03_" + tableName;
+        final String view04 = "v04_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                + "PRIMARY KEY(COL6)) AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            allStmtExecuted = true;
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testSchemaViewIndexWithChildViewExtendedPk2() {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+            stmt.execute("CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                + "PRIMARY KEY(COL6)) AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute("CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                + "SELECT * FROM " + view01 + " WHERE VCOL1 = 'vcol4'");
+            allStmtExecuted = true;
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            fail();
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testTenantSchemaViewIndexWithChildViewExtendedPk2() throws Exception {
+        final String tableName = generateUniqueName();
+        final String schemaName1 = generateUniqueName();
+        final String schemaName2 = generateUniqueName();
+        final String schemaName3 = generateUniqueName();
+        final String schemaName4 = generateUniqueName();
+        final String schemaName5 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName1, tableName);
+        final String view01 = SchemaUtil.getTableName(schemaName2, "v01_" + tableName);
+        final String view02 = SchemaUtil.getTableName(schemaName3, "v02_" + tableName);
+        final String view03 = SchemaUtil.getTableName(schemaName4, "v03_" + tableName);
+        final String view04 = SchemaUtil.getTableName(schemaName5, "v04_" + tableName);
+        final String index_view01 = "idx_v01_" + tableName;
+        boolean allStmtExecuted = false;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT "
+                + "NULL, COL3 VARCHAR, COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(TENANT_ID, "
+                + "COL1, COL2)) MULTI_TENANT = true");
+
+            try (Connection tenantConn = getTenantConnection(TENANT_ID)) {
+                final Statement tenantStmt = tenantConn.createStatement();
+
+                stmt.execute("CREATE VIEW " + view01
+                    + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1))"
+                    + " AS SELECT * FROM " + fullTableName + " WHERE COL1 = 'col1'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR CONSTRAINT pk "
+                        + "PRIMARY KEY(VCOL2)) AS SELECT * FROM " + view01
+                        + " WHERE VCOL1 = 'vcol1'");
+                tenantStmt.execute("CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) "
+                    + "AS SELECT * FROM " + view01 + " WHERE VCOL1 = 'col3'");
+                tenantStmt.execute(
+                    "CREATE VIEW " + view04 + " (VCOL3 CHAR(10), COL6 VARCHAR PRIMARY KEY) AS "
+                        + "SELECT * FROM " + view02 + " WHERE VCOL1 = 'vcol4'");
+                allStmtExecuted = true;
+                stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                    + "(COL1, COL2, COL3)");
+                fail();
+            }
+        } catch (SQLException e) {
+            try {
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getErrorCode(),
+                    e.getErrorCode());
+                assertEquals(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getSQLState(),
+                    e.getSQLState());
+                assertTrue(e.getMessage()
+                    .contains(CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK.getMessage()));
+            } catch (AssertionError ae) {
+                LOGGER.error("Exception: ", e);
+                throw ae;
+            }
+        }
+        assertTrue("All statements could not be executed", allStmtExecuted);
+    }
+
+    @Test
+    public void testViewIndexWithChildViewExtendedPk3() throws Exception {
+        final String tableName = generateUniqueName();
+        final String view01 = "v01_" + tableName;
+        final String view02 = "v02_" + tableName;
+        final String view03 = "v03_" + tableName;
+        final String view04 = "v04_" + tableName;
+        final String index_view01 = "idx_v01_" + tableName;
+        final String index_view02 = "idx_v02_" + tableName;
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            final Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE " + tableName
+                + " (COL1 CHAR(10) NOT NULL, COL2 CHAR(5) NOT NULL, COL3 VARCHAR,"
+                + " COL4 VARCHAR CONSTRAINT pk PRIMARY KEY(COL1, COL2))");
+            stmt.execute("CREATE VIEW " + view01
+                + " (VCOL1 CHAR(8) NOT NULL, COL5 VARCHAR CONSTRAINT pk PRIMARY KEY(VCOL1)) "
+                + "AS SELECT * FROM " + tableName + " WHERE COL1 = 'col1'");
+            stmt.execute(
+                "CREATE VIEW " + view02 + " (VCOL2 CHAR(10), COL6 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'vcol1'");
+            stmt.execute("CREATE INDEX " + index_view02 + " ON " + view02 + " (COL6) INCLUDE "
+                + "(COL1, COL2, COL3)");
+            stmt.execute(
+                "CREATE VIEW " + view03 + " (VCOL3 CHAR(8), COL7 VARCHAR) " + "AS SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'col3'");
+            stmt.execute(
+                "CREATE VIEW " + view04 + " (VCOL2 CHAR(10), COL6 VARCHAR) AS " + "SELECT * FROM "
+                    + view01 + " WHERE VCOL1 = 'vcol4'");
+            stmt.execute("CREATE INDEX " + index_view01 + " ON " + view01 + " (COL5) INCLUDE "
+                + "(COL1, COL2, COL3)");
+        }
+    }
+
+}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
index 77b3ec7ebd..7dba075b8b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
@@ -211,8 +211,9 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
     }
 
     private SchemaBuilder createLevel2TenantViewWithGlobalLevelTTL(
-            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
-            throws Exception {
+            TenantViewOptions tenantViewOptions,
+            TenantViewIndexOptions tenantViewIndexOptions,
+            boolean allowIndex) throws Exception {
         // Define the test schema.
         // 1. Table with columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. GlobalView with columns => (ID, COL4, COL5, COL6), PK => (ID)
@@ -238,10 +239,20 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
         if (tenantViewIndexOptions != null) {
             tenantViewIndexOverrideOptions = tenantViewIndexOptions;
         }
-        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithOverrideOptions)
-                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        if (allowIndex) {
+            schemaBuilder.withTableOptions(tableOptions)
+                    .withGlobalViewOptions(globalViewOptions)
+                    .withGlobalViewIndexOptions(globalViewIndexOptions)
+                    .withTenantViewOptions(tenantViewWithOverrideOptions)
+                    .withTenantViewIndexOptions(tenantViewIndexOverrideOptions)
+                    .buildWithNewTenant();
+        } else {
+            schemaBuilder.withTableOptions(tableOptions)
+                    .withGlobalViewOptions(globalViewOptions)
+                    .withTenantViewOptions(tenantViewWithOverrideOptions)
+                    .withTenantViewIndexOptions(tenantViewIndexOverrideOptions)
+                    .buildWithNewTenant();
+        }
         return schemaBuilder;
     }
 
@@ -259,10 +270,6 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
 
         GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults();
 
-        SchemaBuilder.GlobalViewIndexOptions globalViewIndexOptions
-                = SchemaBuilder.GlobalViewIndexOptions.withDefaults();
-        globalViewIndexOptions.setLocal(false);
-
         TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
         // Phoenix TTL is set to 300s => 300000 ms
         tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=300");
@@ -274,7 +281,6 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
             tenantViewIndexOverrideOptions = tenantViewIndexOptions;
         }
         schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
                 .withTenantViewOptions(tenantViewWithOverrideOptions)
                 .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
         return schemaBuilder;
@@ -400,7 +406,8 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
     public void testPhoenixTTLForLevelTwoView() throws Exception {
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
-        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null,
+                false);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
         String schemaName = stripQuotes(
@@ -409,25 +416,18 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
                 SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
         String tenantViewName = stripQuotes(
                 SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnGlobalViewName = String.format("IDX_%s", globalViewName);
-        String indexOnTenantViewName = String
-                .format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
-        // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
+        // Expected 3 rows - one for GlobalView, one for TenantView and one for tenant view index.
         // Since the PHOENIX_TTL property values are being set,
         // we expect the view header columns to show up in regular scans too.
         assertViewHeaderRowsHavePhoenixTTLRelatedCells(
-                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 4);
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 3);
         assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName,
                 PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName,
-                PTableType.INDEX.getSerializedValue(), 300000);
         // Since the PHOENIX_TTL property values are not being overridden,
         // we expect the TTL value to be same as the global view.
         assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
                 PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
-                PTableType.INDEX.getSerializedValue(), 300000);
     }
 
     @Test
@@ -515,7 +515,7 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
             // Phoenix TTL is set to 120s => 120000 ms
             tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
             final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(
-                    tenantViewWithOverrideOptions, null);
+                    tenantViewWithOverrideOptions, null, false);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
@@ -528,7 +528,8 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         // Phoenix TTL is set to 300s
-        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null,
+                false);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
         String schemaName = stripQuotes(
@@ -537,25 +538,18 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
                 SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
         String tenantViewName = stripQuotes(
                 SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnGlobalViewName = String.format("IDX_%s", globalViewName);
-        String indexOnTenantViewName = String
-                .format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
-        // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
+        // Expected 3 rows - one for GlobalView, one for TenantView and tenant view index.
         // Since the PHOENIX_TTL property values are being set,
         // we expect the view header columns to show up in regular scans too.
         assertViewHeaderRowsHavePhoenixTTLRelatedCells(
-                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 4);
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 3);
         assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName,
                 PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName,
-                PTableType.INDEX.getSerializedValue(), 300000);
         // Since the PHOENIX_TTL property values are not being overridden,
         // we expect the TTL value to be same as the global view.
         assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
                 PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
-                PTableType.INDEX.getSerializedValue(), 300000);
 
         String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
         try (Connection connection = DriverManager.getConnection(tenantURL)) {
@@ -667,7 +661,8 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
     public void testResetPhoenixTTL() throws Exception {
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
-        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null,
+                false);
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
         String schemaName = stripQuotes(
                 SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
@@ -688,11 +683,11 @@ public class ViewTTLIT extends ParallelStatsDisabledIT {
             }
         }
 
-        // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
+        // Expected 3 rows - one for GlobalView, one for TenantView and tenant view index.
         // Since the PHOENIX_TTL property values for global view are being reset,
         // we expect the view header columns value to be set to zero.
         assertViewHeaderRowsHavePhoenixTTLRelatedCells(
-                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 4);
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 3);
         assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName,
                 PTableType.VIEW.getSerializedValue(), 0);
         assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index 3986ea6607..5ca54662c4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -63,12 +63,14 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Iterators;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -176,6 +178,7 @@ public class CreateTableCompiler {
                 validateCreateViewCompilation(connection, parentToBe,
                     columnDefs, pkConstraint);
             }
+            verifyIfAnyParentHasIndexesAndViewExtendsPk(parentToBe, columnDefs, pkConstraint);
         }
         final ViewType viewType = viewTypeToBe;
         final String viewStatement = viewStatementToBe;
@@ -207,6 +210,46 @@ public class CreateTableCompiler {
             viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, connection);
     }
 
+    /**
+     * If any of the parent table/view has indexes in the parent hierarchy, and the current
+     * view under creation extends the primary key of the parent, throw error.
+     *
+     * @param parentToBe parent table/view of the current view under creation.
+     * @param columnDefs list of column definitions.
+     * @param pkConstraint primary key constraint.
+     * @throws SQLException if the view extends primary key and one of the parent view/table has
+     * indexes in the parent hierarchy.
+     */
+    private void verifyIfAnyParentHasIndexesAndViewExtendsPk(PTable parentToBe,
+                                                             List<ColumnDef> columnDefs,
+                                                             PrimaryKeyConstraint pkConstraint)
+            throws SQLException {
+        if (viewExtendsParentPk(columnDefs, pkConstraint)) {
+            PTable table = parentToBe;
+            while (table != null) {
+                if (table.getIndexes().size() > 0) {
+                    throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode
+                                    .VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES)
+                            .build()
+                            .buildException();
+                }
+                if (table.getType() != PTableType.VIEW) {
+                    return;
+                }
+                String schemaName = table.getParentSchemaName().getString();
+                String tableName = table.getParentTableName().getString();
+                try {
+                    table = PhoenixRuntime.getTable(
+                            statement.getConnection(),
+                            SchemaUtil.getTableName(schemaName, tableName));
+                } catch (TableNotFoundException e) {
+                    table = null;
+                }
+            }
+        }
+    }
+
     /**
      * Validate View creation compilation.
      * 1. If view creation syntax does not specify primary key, the method
@@ -253,6 +296,24 @@ public class CreateTableCompiler {
         }
     }
 
+    /**
+     * Returns true if the view extends the primary key of the parent table/view, returns false
+     * otherwise.
+     *
+     * @param columnDefs column def list.
+     * @param pkConstraint primary key constraint.
+     * @return true if the view extends the primary key of the parent table/view, false otherwise.
+     */
+    private boolean viewExtendsParentPk(
+            final List<ColumnDef> columnDefs,
+            final PrimaryKeyConstraint pkConstraint) {
+        if (pkConstraint.getColumnNames().size() > 0) {
+            return true;
+        } else {
+            return columnDefs.stream().anyMatch(ColumnDef::isPK);
+        }
+    }
+
     public static class ColumnTrackingExpressionCompiler extends ExpressionCompiler {
         private final BitSet isColumnReferenced;
         
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 3f4d5168af..5796562566 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -354,6 +354,10 @@ public enum SQLExceptionCode {
             + PhoenixDatabaseMetaData.PHOENIX_TTL + " property on an view when parent/child view has PHOENIX_TTL set,"),
     CHANGE_DETECTION_SUPPORTED_FOR_TABLES_AND_VIEWS_ONLY(10954, "44A36",
         CHANGE_DETECTION_ENABLED + " is only supported on tables and views"),
+    CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK(10955, "44A37", "Index can be created "
+            + "only if none of the child views extends primary key"),
+    VIEW_CANNOT_EXTEND_PK_WITH_PARENT_INDEXES(10956, "44A38", "View can extend parent primary key"
+            + " only if none of the parents have indexes in the parent hierarchy"),
 
     /** Sequence related */
     SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 9bfaf070d8..ada35cc3e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -423,6 +423,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String SYSTEM_CHILD_LINK_TABLE = "CHILD_LINK";
     public static final String SYSTEM_CHILD_LINK_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CHILD_LINK_TABLE);
     public static final byte[] SYSTEM_CHILD_LINK_NAME_BYTES = Bytes.toBytes(SYSTEM_CHILD_LINK_NAME);
+    public static final byte[] SYSTEM_CHILD_LINK_NAMESPACE_BYTES =
+        SchemaUtil.getPhysicalTableName(SYSTEM_CHILD_LINK_NAME_BYTES, true).getName();
     public static final TableName SYSTEM_LINK_HBASE_TABLE_NAME = TableName.valueOf(SYSTEM_CHILD_LINK_NAME);
 
     public static final String SYSTEM_TASK_TABLE = "TASK";
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 46edad7ce9..4934608bd6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -63,7 +63,7 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
     
     @Override
-    protected ConnectionQueryServices getDelegate() {
+    public ConnectionQueryServices getDelegate() {
         return (ConnectionQueryServices)super.getDelegate();
     }
     
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
index e58aa5f5e5..ac241e0b29 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
@@ -40,8 +40,8 @@ public class DelegateQueryServices implements QueryServices {
     public DelegateQueryServices(QueryServices queryServices) {
         parent = queryServices;
     }
-    
-    protected QueryServices getDelegate() {
+
+    public QueryServices getDelegate() {
         return parent;
     }
     
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index af2c518a1c..e477f28cf4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -435,6 +435,17 @@ public interface QueryServices extends SQLCloseable {
      */
     String SERVER_MERGE_FOR_UNCOVERED_INDEX = "phoenix.query.global.server.merge.enable";
 
+    /**
+     * Param to determine whether client can disable validation to figure out if any of the
+     * descendent views extend primary key of their parents. Since this is a bit of
+     * expensive call, we can opt in to disable it. By default, this check will always be performed
+     * while creating index (PHOENIX-7067) on any table or view. This config can be used for
+     * disabling other subtree validation purpose as well.
+     */
+    String DISABLE_VIEW_SUBTREE_VALIDATION = "phoenix.disable.view.subtree.validation";
+
+    boolean DEFAULT_DISABLE_VIEW_SUBTREE_VALIDATION = false;
+
     /**
      * Get executor service used for parallel scans
      */
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index b4ce6c2351..fb33efbada 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -81,6 +81,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STREAMING_TOPIC_NA
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYNC_INDEX_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAMESPACE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_TASK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
@@ -98,6 +100,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.SYSTEM_SCHEMA_NAME;
+import static org.apache.phoenix.query.QueryServices.DEFAULT_DISABLE_VIEW_SUBTREE_VALIDATION;
+import static org.apache.phoenix.query.QueryServices.DISABLE_VIEW_SUBTREE_VALIDATION;
 import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE;
 import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSet;
 import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
@@ -153,6 +157,10 @@ import java.util.Set;
 import java.util.HashSet;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.phoenix.coprocessor.TableInfo;
+import org.apache.phoenix.query.ConnectionlessQueryServicesImpl;
+import org.apache.phoenix.query.DelegateQueryServices;
 import org.apache.phoenix.schema.task.SystemTaskParams;
 import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.HConstants;
@@ -1596,6 +1604,12 @@ public class MetaDataClient {
                 }
             }
 
+            Configuration config = connection.getQueryServices().getConfiguration();
+            if (!connection.getQueryServices().getProps()
+                .getBoolean(DISABLE_VIEW_SUBTREE_VALIDATION,
+                    DEFAULT_DISABLE_VIEW_SUBTREE_VALIDATION)) {
+                verifyIfDescendentViewsExtendPk(dataTable, config);
+            }
             // for view indexes
             if (dataTable.getType() == PTableType.VIEW) {
                 String physicalName = dataTable.getPhysicalName().getString();
@@ -1620,7 +1634,6 @@ public class MetaDataClient {
                 }
             }
 
-            Configuration config = connection.getQueryServices().getConfiguration();
             long threshold = Long.parseLong(config.get(QueryServices.CLIENT_INDEX_ASYNC_THRESHOLD));
 
             if (threshold > 0 && !statement.isAsync()) {
@@ -1701,6 +1714,65 @@ public class MetaDataClient {
         return buildIndex(table, tableRef);
     }
 
+    /**
+     * Go through all the descendent views from the child view hierarchy and find if any of the
+     * descendent views extends the primary key, throw error.
+     *
+     * @param tableOrView view or table on which the index is being created.
+     * @param config the configuration.
+     * @throws SQLException if any of the descendent views extends pk or if something goes wrong
+     * while querying descendent view hierarchy.
+     */
+    private void verifyIfDescendentViewsExtendPk(PTable tableOrView, Configuration config)
+        throws SQLException {
+        if (connection.getQueryServices() instanceof ConnectionlessQueryServicesImpl) {
+            return;
+        }
+        if (connection.getQueryServices() instanceof DelegateQueryServices) {
+            DelegateQueryServices services = (DelegateQueryServices) connection.getQueryServices();
+            if (services.getDelegate() instanceof ConnectionlessQueryServicesImpl) {
+                return;
+            }
+        }
+        byte[] systemChildLinkTable = SchemaUtil.isNamespaceMappingEnabled(null, config) ?
+            SYSTEM_CHILD_LINK_NAMESPACE_BYTES :
+            SYSTEM_CHILD_LINK_NAME_BYTES;
+        try (Table childLinkTable =
+                     connection.getQueryServices().getTable(systemChildLinkTable)) {
+            byte[] tenantId = connection.getTenantId() == null ? null
+                    : connection.getTenantId().getBytes();
+            byte[] schemaNameBytes = tableOrView.getSchemaName().getBytes();
+            byte[] viewOrTableName = tableOrView.getTableName().getBytes();
+            Pair<List<PTable>, List<TableInfo>> descViews =
+                    ViewUtil.findAllDescendantViews(
+                            childLinkTable,
+                            config,
+                            tenantId,
+                            schemaNameBytes,
+                            viewOrTableName,
+                            HConstants.LATEST_TIMESTAMP,
+                            false);
+            List<PTable> legitimateChildViews = descViews.getFirst();
+            int dataTableOrViewPkCols = tableOrView.getPKColumns().size();
+            if (legitimateChildViews != null && legitimateChildViews.size() > 0) {
+                for (PTable childView : legitimateChildViews) {
+                    if (childView.getPKColumns().size() > dataTableOrViewPkCols) {
+                        LOGGER.error("Creation of view index not allowed as child view {}"
+                                + " extends pk", childView.getName());
+                        throw new SQLExceptionInfo.Builder(
+                                SQLExceptionCode
+                                        .CANNOT_CREATE_INDEX_CHILD_VIEWS_EXTEND_PK)
+                                .build()
+                                .buildException();
+                    }
+                }
+            }
+        } catch (IOException e) {
+            LOGGER.error("Error while retrieving descendent views", e);
+            throw new SQLException(e);
+        }
+    }
+
     public MutationState dropSequence(DropSequenceStatement statement) throws SQLException {
         Long scn = connection.getSCN();
         long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;