You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by GitBox <gi...@apache.org> on 2020/10/26 19:31:44 UTC

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #916: PHOENIX-6171 Child views should not be allowed to override the parent view PHOENIX_TTL attribute.

ChinmaySKulkarni commented on a change in pull request #916:
URL: https://github.com/apache/phoenix/pull/916#discussion_r509545181



##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -61,22 +62,31 @@
     private static final Logger LOGGER = LoggerFactory.getLogger(ViewTTLIT.class);
     private static final String ORG_ID_FMT = "00D0x000%s";
     private static final String ID_FMT = "00A0y000%07d";
-    private static final String PHOENIX_TTL_HEADER_SQL =  "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
-            + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+    private static final String
+            PHOENIX_TTL_HEADER_SQL =
+            "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
+                    + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+
+    private static final String
+            ALTER_PHOENIX_TTL_SQL =
+            "ALTER VIEW \"%s\".\"%s\" set PHOENIX_TTL=%s";
 
-    private static final String ALTER_PHOENIX_TTL_SQL = "ALTER VIEW %s.%s set PHOENIX_TTL=%d";
+    private static final String

Review comment:
       nit: Modify formatting to avoid extra line

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -61,22 +62,31 @@
     private static final Logger LOGGER = LoggerFactory.getLogger(ViewTTLIT.class);
     private static final String ORG_ID_FMT = "00D0x000%s";
     private static final String ID_FMT = "00A0y000%07d";
-    private static final String PHOENIX_TTL_HEADER_SQL =  "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
-            + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+    private static final String
+            PHOENIX_TTL_HEADER_SQL =
+            "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
+                    + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+
+    private static final String
+            ALTER_PHOENIX_TTL_SQL =

Review comment:
       nit: Modify formatting to avoid extra line

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2639,6 +2646,20 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
                 }
             }
+            if (settingNewPhoenixTTLAttribute(tableMetadata, PHOENIX_TTL_BYTES)) {

Review comment:
       In fact, since we only allow setting PHOENIX_TTL on views we should enter this `if` block only if PTableType== VIEW (maybe add it inside the `settingNewPhoenixTTLAttribute()` method itself so we can change it in the future if required).

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2639,6 +2646,20 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
                 }
             }
+            if (settingNewPhoenixTTLAttribute(tableMetadata, PHOENIX_TTL_BYTES)) {
+                // Disallow if the parent has PHOENIX_TTL set.
+                if (parentTable != null &&  parentTable.getPhoenixTTL() != PHOENIX_TTL_NOT_DEFINED) {
+                    isSchemaMutationAllowed = false;
+                }
+
+                // Since we do not allow propagation of PHOENIX_TTL values during ALTER for now.
+                // If a child view exists and the parent previously had a PHOENIX_TTL value set then that

Review comment:
       We aren't checking the "and the parent previously had a PHOENIX_TTL value set " part. Shouldn't we do that too?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/AddColumnMutator.java
##########
@@ -59,6 +59,8 @@
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHOENIX_TTL_BYTES;

Review comment:
       Unused imports?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2639,6 +2646,20 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
                 }
             }
+            if (settingNewPhoenixTTLAttribute(tableMetadata, PHOENIX_TTL_BYTES)) {
+                // Disallow if the parent has PHOENIX_TTL set.
+                if (parentTable != null &&  parentTable.getPhoenixTTL() != PHOENIX_TTL_NOT_DEFINED) {
+                    isSchemaMutationAllowed = false;
+                }
+
+                // Since we do not allow propagation of PHOENIX_TTL values during ALTER for now.
+                // If a child view exists and the parent previously had a PHOENIX_TTL value set then that
+                // implies that the child view too has a valid PHOENIX_TTL (non zero).
+                // In this case we do not allow for ALTER of the parent view PHOENIX_TTL value.
+                if (!childViews.isEmpty()) {

Review comment:
       What if we're setting the TTL for the first time for a parent view i.e. it was PHOENIX_TTL_NOT_DEFINED first and now we're setting it for the first time ever. This check will disallow that too. Is that intended?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -61,22 +62,31 @@
     private static final Logger LOGGER = LoggerFactory.getLogger(ViewTTLIT.class);
     private static final String ORG_ID_FMT = "00D0x000%s";
     private static final String ID_FMT = "00A0y000%07d";
-    private static final String PHOENIX_TTL_HEADER_SQL =  "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
-            + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+    private static final String
+            PHOENIX_TTL_HEADER_SQL =
+            "SELECT PHOENIX_TTL FROM SYSTEM.CATALOG "
+                    + "WHERE %s AND TABLE_SCHEM = '%s' AND TABLE_NAME = '%s' AND TABLE_TYPE = '%s'";
+
+    private static final String
+            ALTER_PHOENIX_TTL_SQL =
+            "ALTER VIEW \"%s\".\"%s\" set PHOENIX_TTL=%s";
 
-    private static final String ALTER_PHOENIX_TTL_SQL = "ALTER VIEW %s.%s set PHOENIX_TTL=%d";
+    private static final String
+            ALTER_SQL_WITH_NO_TTL =
+            "ALTER VIEW \"%s\".\"%s\" ADD IF NOT EXISTS %s CHAR(10)";
 
     // Scans the HBase rows directly for the view ttl related header rows column and asserts
-    private void assertViewHeaderRowsHavePhoenixTTLRelatedCells(String schemaName, long minTimestamp,
-            boolean rawScan, int expectedRows) throws IOException, SQLException {
+    private void assertViewHeaderRowsHavePhoenixTTLRelatedCells(String schemaName,
+            long minTimestamp, boolean rawScan, int expectedRows) throws IOException, SQLException {
 
         FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
-        RowFilter schemaNameFilter = new RowFilter(
-                CompareFilter.CompareOp.EQUAL,
-                new SubstringComparator(schemaName)
-        );
-        QualifierFilter phoenixTTLQualifierFilter = new QualifierFilter(CompareFilter.CompareOp.EQUAL,
-                new BinaryComparator(PhoenixDatabaseMetaData.PHOENIX_TTL_BYTES));
+        RowFilter
+                schemaNameFilter =
+                new RowFilter(CompareFilter.CompareOp.EQUAL, new SubstringComparator(schemaName));
+        QualifierFilter

Review comment:
       Similar here. Can you please check if your IDE has the Phoenix IDE settings?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2925,6 +2954,23 @@ private void clearRemoteTableFromCache(long clientTimeStamp, byte[] schemaName,
         }
     }
 
+    // Checks whether a non-zero PHOENIX_TTL value is being set.
+    private boolean settingNewPhoenixTTLAttribute(List<Mutation> tableMetadata, byte[] phoenixTtlBytes) {

Review comment:
       Can you rename this to `settingNewPhoenixTTLAttributeForView()` to indicate it only applies to views and then also add the check for the same inside the method?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -119,283 +137,556 @@ private String stripQuotes(String name) {
         return name.replace("\"", "");
     }
 
+    private SchemaBuilder createLevel2TenantViewWithGlobalLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults();
+        // Phoenix TTL is set to 300s => 300000 ms
+        globalViewOptions.setTableProps("PHOENIX_TTL=300");
+
+        SchemaBuilder.GlobalViewIndexOptions
+                globalViewIndexOptions =
+                SchemaBuilder.GlobalViewIndexOptions.withDefaults();
+        globalViewIndexOptions.setLocal(false);
+
+        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel2TenantViewWithTenantLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        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");
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel1TenantView(TenantViewOptions tenantViewOptions,
+            TenantViewIndexOptions tenantViewIndexOptions) throws Exception {
+        // Define the test schema.
+        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
+        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        TenantViewOptions tenantViewOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+
+        schemaBuilder.withTableOptions(tableOptions)
+                .withTenantViewOptions(tenantViewOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildNewView();
+        return schemaBuilder;
+    }
 
     /**
      * -----------------
      * Test methods
      * -----------------
      */
 
-    @Test
-    public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
+    @Test public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
 
-        long startTime = System.currentTimeMillis();
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-        schemaBuilder
-                .withTableDefaults()
-                .withGlobalViewDefaults()
-                .build();
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+        schemaBuilder.withTableDefaults().withGlobalViewDefaults().build();
 
         // Expected 2 rows - one for Table and GlobalView each.
-        // Since the PHOENIX_TTL property values are not being set, we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
+        // Since the PHOENIX_TTL property values are not being set,
+        // we expect the view header columns to show up in raw scans only.
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
     }
 
-
-
-    @Test
-    public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
+    @Test public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
 
         TableOptions tableOptions = TableOptions.withDefaults();
         tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true,TTL=100");
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         tenantViewOptions.setTableProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewOptions(tenantViewOptions)
+            schemaBuilder.withTableOptions(tableOptions).withTenantViewOptions(tenantViewOptions)
                     .buildNewView();
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL
+                    .getErrorCode(), e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLWithViewIndexFails() throws Exception {
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithViewIndexFails() throws Exception {
 
         TenantViewIndexOptions tenantViewIndexOptions = TenantViewIndexOptions.withDefaults();
         tenantViewIndexOptions.setIndexProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewDefaults()
-                    .withTenantViewIndexOptions(tenantViewIndexOptions)
-                    .buildNewView();
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel1TenantView(null, tenantViewIndexOptions);
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(),
+                    e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLForLevelOneView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForLevelOneView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Phoenix TTL is set to 120s => 120000 ms
         tenantViewOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
 
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
 
     }
 
-    @Test
-    public void testPhoenixTTLForLevelTwoView() throws Exception {
-        long startTime = System.currentTimeMillis();
+    @Test public void testPhoenixTTLForLevelTwoView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        // 3. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions
-                globalViewOptions = PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions.withDefaults();
-        // Phoenix TTL is set to 300s => 300000 ms
-        globalViewOptions.setTableProps("PHOENIX_TTL=300");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions
-                globalViewIndexOptions =
-                PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions.withDefaults();
-        globalViewIndexOptions.setLocal(false);
-
-        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
-        // Phoenix TTL is set to 120s => 120000 ms
-        tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
-        // 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);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
-
-        // Without override
-        startTime = System.currentTimeMillis();
-
-        TenantViewOptions tenantViewWithoutOverrideOptions = TenantViewOptions.withDefaults();
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithoutOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
-
-        tenantId = schemaBuilder.getDataOptions().getTenantId();
-        schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        indexOnGlobalViewName = String.format("IDX_%s", globalViewName);
-        indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
-
-
-        // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, 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);
+        // 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);
+        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
-    public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=NONE
         tenantViewOptions.setTableProps("PHOENIX_TTL=0");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
     }
 
-    @Test
-    public void testPhoenixTTLWithAlterView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithAlterView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=0
         tenantViewOptions.setTableProps("PHOENIX_TTL=NONE");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
         String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
         try (Connection connection = DriverManager.getConnection(tenantURL)) {
-            Statement stmt = connection.createStatement();
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, "120");
+                stmt.execute(sql);
+            }
+        }
+
+        // Expected 2 rows - one for TenantView and ViewIndex each.
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
+
+    }
+
+    @Test public void testCreateViewWithParentPhoenixTTLFails() throws Exception {
+        try {
+            TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
             // Phoenix TTL is set to 120s => 120000 ms
-            String sql = String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, 120);
-            stmt.execute(sql);
+            tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel2TenantViewWithGlobalLevelTTL(tenantViewWithOverrideOptions, null);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
         }
+    }
+
+    @Test public void testAlterViewWithParentPhoenixTTLFails() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String

Review comment:
       nit: Same formatting issues here potentially.

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2639,6 +2646,20 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
                 }
             }
+            if (settingNewPhoenixTTLAttribute(tableMetadata, PHOENIX_TTL_BYTES)) {

Review comment:
       Circumvent this check if the tableType != VIEW to prevent unnecessary iteration of the list of mutations in non-view cases.

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -119,283 +137,556 @@ private String stripQuotes(String name) {
         return name.replace("\"", "");
     }
 
+    private SchemaBuilder createLevel2TenantViewWithGlobalLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults();
+        // Phoenix TTL is set to 300s => 300000 ms
+        globalViewOptions.setTableProps("PHOENIX_TTL=300");
+
+        SchemaBuilder.GlobalViewIndexOptions
+                globalViewIndexOptions =
+                SchemaBuilder.GlobalViewIndexOptions.withDefaults();
+        globalViewIndexOptions.setLocal(false);
+
+        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel2TenantViewWithTenantLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        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");
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel1TenantView(TenantViewOptions tenantViewOptions,
+            TenantViewIndexOptions tenantViewIndexOptions) throws Exception {
+        // Define the test schema.
+        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
+        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        TenantViewOptions tenantViewOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+
+        schemaBuilder.withTableOptions(tableOptions)
+                .withTenantViewOptions(tenantViewOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildNewView();
+        return schemaBuilder;
+    }
 
     /**
      * -----------------
      * Test methods
      * -----------------
      */
 
-    @Test
-    public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
+    @Test public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
 
-        long startTime = System.currentTimeMillis();
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-        schemaBuilder
-                .withTableDefaults()
-                .withGlobalViewDefaults()
-                .build();
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+        schemaBuilder.withTableDefaults().withGlobalViewDefaults().build();
 
         // Expected 2 rows - one for Table and GlobalView each.
-        // Since the PHOENIX_TTL property values are not being set, we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
+        // Since the PHOENIX_TTL property values are not being set,
+        // we expect the view header columns to show up in raw scans only.
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
     }
 
-
-
-    @Test
-    public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
+    @Test public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
 
         TableOptions tableOptions = TableOptions.withDefaults();
         tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true,TTL=100");
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         tenantViewOptions.setTableProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewOptions(tenantViewOptions)
+            schemaBuilder.withTableOptions(tableOptions).withTenantViewOptions(tenantViewOptions)
                     .buildNewView();
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL
+                    .getErrorCode(), e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLWithViewIndexFails() throws Exception {
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithViewIndexFails() throws Exception {
 
         TenantViewIndexOptions tenantViewIndexOptions = TenantViewIndexOptions.withDefaults();
         tenantViewIndexOptions.setIndexProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewDefaults()
-                    .withTenantViewIndexOptions(tenantViewIndexOptions)
-                    .buildNewView();
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel1TenantView(null, tenantViewIndexOptions);
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(),
+                    e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLForLevelOneView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForLevelOneView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Phoenix TTL is set to 120s => 120000 ms
         tenantViewOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
 
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
 
     }
 
-    @Test
-    public void testPhoenixTTLForLevelTwoView() throws Exception {
-        long startTime = System.currentTimeMillis();
+    @Test public void testPhoenixTTLForLevelTwoView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        // 3. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions
-                globalViewOptions = PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions.withDefaults();
-        // Phoenix TTL is set to 300s => 300000 ms
-        globalViewOptions.setTableProps("PHOENIX_TTL=300");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions
-                globalViewIndexOptions =
-                PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions.withDefaults();
-        globalViewIndexOptions.setLocal(false);
-
-        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
-        // Phoenix TTL is set to 120s => 120000 ms
-        tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
-        // 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);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
-
-        // Without override
-        startTime = System.currentTimeMillis();
-
-        TenantViewOptions tenantViewWithoutOverrideOptions = TenantViewOptions.withDefaults();
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithoutOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
-
-        tenantId = schemaBuilder.getDataOptions().getTenantId();
-        schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        indexOnGlobalViewName = String.format("IDX_%s", globalViewName);
-        indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
-
-
-        // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, 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);
+        // 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);
+        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
-    public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=NONE
         tenantViewOptions.setTableProps("PHOENIX_TTL=0");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
     }
 
-    @Test
-    public void testPhoenixTTLWithAlterView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithAlterView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=0
         tenantViewOptions.setTableProps("PHOENIX_TTL=NONE");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
         String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
         try (Connection connection = DriverManager.getConnection(tenantURL)) {
-            Statement stmt = connection.createStatement();
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, "120");
+                stmt.execute(sql);
+            }
+        }
+
+        // Expected 2 rows - one for TenantView and ViewIndex each.
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
+
+    }
+
+    @Test public void testCreateViewWithParentPhoenixTTLFails() throws Exception {
+        try {
+            TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
             // Phoenix TTL is set to 120s => 120000 ms
-            String sql = String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, 120);
-            stmt.execute(sql);
+            tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel2TenantViewWithGlobalLevelTTL(tenantViewWithOverrideOptions, null);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
         }
+    }
+
+    @Test public void testAlterViewWithParentPhoenixTTLFails() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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.
+        // 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);
+        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)) {
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, "120");
+                stmt.execute(sql);
+                fail();
+            }
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
+        }
+    }
+
+    @Test public void testAlterViewWithChildLevelPhoenixTTLFails() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithTenantLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 300000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 300000);
+
+        try (Connection connection = DriverManager.getConnection(getUrl())) {
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, globalViewName, "120");
+                stmt.execute(sql);
+                fail();
+            }
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
+        }
+    }
+
+    @Test public void testAlterViewWithNoPhoenixTTLSucceed() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithTenantLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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 2 rows - one for TenantView and ViewIndex each.
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 300000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 300000);
+
+        // ALTER global view
+        try (Connection connection = DriverManager.getConnection(getUrl())) {
+            try (Statement stmt = connection.createStatement()) {
+                String
+                        sql =
+                        String.format(ALTER_SQL_WITH_NO_TTL, schemaName, globalViewName, "COL_30");
+                stmt.execute(sql);
+            }
+        } catch (SQLException e) {
+            fail();
+        }
+
+        // ALTER tenant view
+        String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
+        try (Connection connection = DriverManager.getConnection(tenantURL)) {
+            try (Statement stmt = connection.createStatement()) {
+                String
+                        sql =
+                        String.format(ALTER_SQL_WITH_NO_TTL, schemaName, tenantViewName, "COL_100");
+                stmt.execute(sql);
+            }
+        } catch (SQLException e) {

Review comment:
       Remove catch block since it's unnecessary.

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java
##########
@@ -119,283 +137,556 @@ private String stripQuotes(String name) {
         return name.replace("\"", "");
     }
 
+    private SchemaBuilder createLevel2TenantViewWithGlobalLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults();
+        // Phoenix TTL is set to 300s => 300000 ms
+        globalViewOptions.setTableProps("PHOENIX_TTL=300");
+
+        SchemaBuilder.GlobalViewIndexOptions
+                globalViewIndexOptions =
+                SchemaBuilder.GlobalViewIndexOptions.withDefaults();
+        globalViewIndexOptions.setLocal(false);
+
+        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel2TenantViewWithTenantLevelTTL(
+            TenantViewOptions tenantViewOptions, TenantViewIndexOptions tenantViewIndexOptions)
+            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)
+        // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        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");
+        if (tenantViewOptions != null) {
+            tenantViewWithOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+        schemaBuilder.withTableOptions(tableOptions).withGlobalViewOptions(globalViewOptions)
+                .withGlobalViewIndexOptions(globalViewIndexOptions)
+                .withTenantViewOptions(tenantViewWithOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildWithNewTenant();
+        return schemaBuilder;
+    }
+
+    private SchemaBuilder createLevel1TenantView(TenantViewOptions tenantViewOptions,
+            TenantViewIndexOptions tenantViewIndexOptions) throws Exception {
+        // Define the test schema.
+        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
+        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+
+        TableOptions tableOptions = TableOptions.withDefaults();
+        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+
+        TenantViewOptions tenantViewOverrideOptions = TenantViewOptions.withDefaults();
+        if (tenantViewOptions != null) {
+            tenantViewOverrideOptions = tenantViewOptions;
+        }
+        TenantViewIndexOptions
+                tenantViewIndexOverrideOptions =
+                TenantViewIndexOptions.withDefaults();
+        if (tenantViewIndexOptions != null) {
+            tenantViewIndexOverrideOptions = tenantViewIndexOptions;
+        }
+
+        schemaBuilder.withTableOptions(tableOptions)
+                .withTenantViewOptions(tenantViewOverrideOptions)
+                .withTenantViewIndexOptions(tenantViewIndexOverrideOptions).buildNewView();
+        return schemaBuilder;
+    }
 
     /**
      * -----------------
      * Test methods
      * -----------------
      */
 
-    @Test
-    public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
+    @Test public void testWithBasicGlobalViewWithNoPhoenixTTLDefined() throws Exception {
 
-        long startTime = System.currentTimeMillis();
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-        schemaBuilder
-                .withTableDefaults()
-                .withGlobalViewDefaults()
-                .build();
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
+        schemaBuilder.withTableDefaults().withGlobalViewDefaults().build();
 
         // Expected 2 rows - one for Table and GlobalView each.
-        // Since the PHOENIX_TTL property values are not being set, we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
+        // Since the PHOENIX_TTL property values are not being set,
+        // we expect the view header columns to show up in raw scans only.
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 2);
     }
 
-
-
-    @Test
-    public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
+    @Test public void testPhoenixTTLWithTableLevelTTLFails() throws Exception {
 
         // Define the test schema.
         // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
         // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
+        final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl());
 
         TableOptions tableOptions = TableOptions.withDefaults();
         tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true,TTL=100");
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         tenantViewOptions.setTableProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewOptions(tenantViewOptions)
+            schemaBuilder.withTableOptions(tableOptions).withTenantViewOptions(tenantViewOptions)
                     .buildNewView();
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL_FOR_TABLE_WITH_TTL
+                    .getErrorCode(), e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLWithViewIndexFails() throws Exception {
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithViewIndexFails() throws Exception {
 
         TenantViewIndexOptions tenantViewIndexOptions = TenantViewIndexOptions.withDefaults();
         tenantViewIndexOptions.setIndexProps("PHOENIX_TTL=1000");
         try {
-            schemaBuilder
-                    .withTableOptions(tableOptions)
-                    .withTenantViewDefaults()
-                    .withTenantViewIndexOptions(tenantViewIndexOptions)
-                    .buildNewView();
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel1TenantView(null, tenantViewIndexOptions);
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.PHOENIX_TTL_SUPPORTED_FOR_VIEWS_ONLY.getErrorCode(),
+                    e.getErrorCode());
         }
     }
 
-    @Test
-    public void testPhoenixTTLForLevelOneView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForLevelOneView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Phoenix TTL is set to 120s => 120000 ms
         tenantViewOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
 
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
 
     }
 
-    @Test
-    public void testPhoenixTTLForLevelTwoView() throws Exception {
-        long startTime = System.currentTimeMillis();
+    @Test public void testPhoenixTTLForLevelTwoView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. GlobalView with default columns => (ID, COL4, COL5, COL6), PK => (ID)
-        // 3. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions
-                globalViewOptions = PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions.withDefaults();
-        // Phoenix TTL is set to 300s => 300000 ms
-        globalViewOptions.setTableProps("PHOENIX_TTL=300");
-
-        PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions
-                globalViewIndexOptions =
-                PhoenixTestBuilder.SchemaBuilder.GlobalViewIndexOptions.withDefaults();
-        globalViewIndexOptions.setLocal(false);
-
-        TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
-        // Phoenix TTL is set to 120s => 120000 ms
-        tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 4 rows - one for GlobalView, one for TenantView and ViewIndex each.
-        // 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);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
-
-        // Without override
-        startTime = System.currentTimeMillis();
-
-        TenantViewOptions tenantViewWithoutOverrideOptions = TenantViewOptions.withDefaults();
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withGlobalViewOptions(globalViewOptions)
-                .withGlobalViewIndexOptions(globalViewIndexOptions)
-                .withTenantViewOptions(tenantViewWithoutOverrideOptions)
-                .withTenantViewIndexDefaults()
-                .buildWithNewTenant();
-
-        tenantId = schemaBuilder.getDataOptions().getTenantId();
-        schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        globalViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityGlobalViewName()));
-        tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        indexOnGlobalViewName = String.format("IDX_%s", globalViewName);
-        indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
-
-
-        // Expected 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, PTableType.VIEW.getSerializedValue(), 300000);
-        assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, PTableType.INDEX.getSerializedValue(), 300000);
-        // Since the PHOENIX_TTL property values are not being overriden, 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);
+        // 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);
+        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
-    public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLForWhenTTLIsZero() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=NONE
         tenantViewOptions.setTableProps("PHOENIX_TTL=0");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
     }
 
-    @Test
-    public void testPhoenixTTLWithAlterView() throws Exception {
-        long startTime = System.currentTimeMillis();
-
-        // Define the test schema.
-        // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP)
-        // 2. Tenant with default columns => (ZID, COL7, COL8, COL9), PK => (ZID)
-        final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl());
-
-        TableOptions tableOptions = TableOptions.withDefaults();
-        tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true");
+    @Test public void testPhoenixTTLWithAlterView() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
 
         TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults();
         // Client can also specify PHOENIX_TTL=0
         tenantViewOptions.setTableProps("PHOENIX_TTL=NONE");
-        schemaBuilder
-                .withTableOptions(tableOptions)
-                .withTenantViewOptions(tenantViewOptions)
-                .withTenantViewIndexDefaults()
-                .buildNewView();
+        final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null);
 
         String tenantId = schemaBuilder.getDataOptions().getTenantId();
-        String schemaName = stripQuotes(SchemaUtil.getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String tenantViewName = stripQuotes(SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
-        String indexOnTenantViewName = String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                tenantViewName =
+                stripQuotes(SchemaUtil
+                        .getTableNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                indexOnTenantViewName =
+                String.format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix()));
 
         // Expected 3 deleted rows - one for Table, one for TenantView and ViewIndex each.
         // Since the PHOENIX_TTL property values are not being set or being set to zero,
         // we expect the view header columns to show up in raw scans only.
-        assertViewHeaderRowsHavePhoenixTTLRelatedCells(schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 0);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 0);
+        assertViewHeaderRowsHavePhoenixTTLRelatedCells(
+                schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 0);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 0);
 
         String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
         try (Connection connection = DriverManager.getConnection(tenantURL)) {
-            Statement stmt = connection.createStatement();
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, "120");
+                stmt.execute(sql);
+            }
+        }
+
+        // Expected 2 rows - one for TenantView and ViewIndex each.
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 120000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 120000);
+
+    }
+
+    @Test public void testCreateViewWithParentPhoenixTTLFails() throws Exception {
+        try {
+            TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults();
             // Phoenix TTL is set to 120s => 120000 ms
-            String sql = String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, 120);
-            stmt.execute(sql);
+            tenantViewWithOverrideOptions.setTableProps("PHOENIX_TTL=120");
+            final SchemaBuilder
+                    schemaBuilder =
+                    createLevel2TenantViewWithGlobalLevelTTL(tenantViewWithOverrideOptions, null);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
         }
+    }
+
+    @Test public void testAlterViewWithParentPhoenixTTLFails() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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.
+        // 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);
+        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)) {
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, tenantViewName, "120");
+                stmt.execute(sql);
+                fail();
+            }
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
+        }
+    }
+
+    @Test public void testAlterViewWithChildLevelPhoenixTTLFails() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithTenantLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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 2 rows - one for TenantView and ViewIndex each.
-        // 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, 2);
-        // Since the PHOENIX_TTL property values are not being overriden, we expect the TTL value to be different from the global view.
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, PTableType.VIEW.getSerializedValue(), 120000);
-        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, PTableType.INDEX.getSerializedValue(), 120000);
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 300000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 300000);
+
+        try (Connection connection = DriverManager.getConnection(getUrl())) {
+            try (Statement stmt = connection.createStatement()) {
+                // Phoenix TTL is set to 120s => 120000 ms
+                String
+                        sql =
+                        String.format(ALTER_PHOENIX_TTL_SQL, schemaName, globalViewName, "120");
+                stmt.execute(sql);
+                fail();
+            }
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_SET_OR_ALTER_PHOENIX_TTL.getErrorCode(),
+                    e.getErrorCode());
+        }
+    }
+
+    @Test public void testAlterViewWithNoPhoenixTTLSucceed() throws Exception {
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+
+        // Phoenix TTL is set to 300s
+        final SchemaBuilder schemaBuilder = createLevel2TenantViewWithTenantLevelTTL(null, null);
+
+        String tenantId = schemaBuilder.getDataOptions().getTenantId();
+        String
+                schemaName =
+                stripQuotes(SchemaUtil
+                        .getSchemaNameFromFullName(schemaBuilder.getEntityTenantViewName()));
+        String
+                globalViewName =
+                stripQuotes(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 2 rows - one for TenantView and ViewIndex each.
+        // 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, 2);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName,
+                PTableType.VIEW.getSerializedValue(), 300000);
+        assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName,
+                PTableType.INDEX.getSerializedValue(), 300000);
+
+        // ALTER global view
+        try (Connection connection = DriverManager.getConnection(getUrl())) {
+            try (Statement stmt = connection.createStatement()) {
+                String
+                        sql =
+                        String.format(ALTER_SQL_WITH_NO_TTL, schemaName, globalViewName, "COL_30");
+                stmt.execute(sql);
+            }
+        } catch (SQLException e) {

Review comment:
       This `catch` block is unnecessary, you can remove it.

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
##########
@@ -2077,6 +2077,13 @@ private PTable createTableInternal(CreateTableStatement statement, byte[][] spli
                     }
                 }
 
+                // Cannot set PHOENIX_TTL if parent has already defined it.
+                if (tableType == VIEW  && parent != null && parent.getPhoenixTTL() != PHOENIX_TTL_NOT_DEFINED) {

Review comment:
       We're doing this check and still doing a similar one on the server. Do we need both?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org