You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/07/02 23:01:32 UTC

[29/47] phoenix git commit: PHOENIX-2055 Allow view with views to add column

PHOENIX-2055 Allow view with views to add column


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

Branch: refs/heads/calcite
Commit: 9c069bd45e6af789f903a5e1f80949de403bbf68
Parents: 7918a3d
Author: Samarth <sa...@salesforce.com>
Authored: Fri Jun 26 16:22:39 2015 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Jun 26 16:22:39 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    | 16 +++--
 .../org/apache/phoenix/end2end/UpgradeIT.java   | 12 ++--
 .../coprocessor/MetaDataEndpointImpl.java       | 71 +++++++++++---------
 .../apache/phoenix/query/QueryConstants.java    |  2 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  6 +-
 5 files changed, 60 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c069bd4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index ae5f940..946aaab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -2129,8 +2129,8 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
     }
     
     @Test
-    public void testAlteringViewThatHasChildViewsNotAllowed() throws Exception {
-        String baseTable = "testAlteringViewThatHasChildViewsNotAllowed";
+    public void testAlteringViewThatHasChildViews() throws Exception {
+        String baseTable = "testAlteringViewThatHasChildViews";
         String childView = "childView";
         String grandChildView = "grandChildView";
         try (Connection conn = DriverManager.getConnection(getUrl())) {
@@ -2167,13 +2167,17 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
                 assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
             }
             
-            // Adding column to view that has child views should fail
+            // Adding column to view that has child views is allowed
             String addColumnToChildView = "ALTER VIEW " + childView + " ADD V5 VARCHAR";
+            conn.createStatement().execute(addColumnToChildView);
+            // V5 column should be visible now for childView
+            conn.createStatement().execute("SELECT V5 FROM " + childView);    
+            
+            // However, column V5 shouldn't have propagated to grandChildView. Not till PHOENIX-2054 is fixed.
             try {
-                conn.createStatement().execute(addColumnToChildView);
-                fail("Adding columns to a view that has child views on it is not allowed");
+                conn.createStatement().execute("SELECT V5 FROM " + grandChildView);
             } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
             }
 
             // dropping column from the grand child view, however, should work.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c069bd4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 886e567..094816c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -20,7 +20,7 @@ package org.apache.phoenix.end2end;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.query.QueryConstants.DIVORCED_VIEW_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.util.UpgradeUtil.SELECT_BASE_COLUMN_COUNT_FROM_HEADER_ROW;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -187,14 +187,14 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             for (int i = 1; i <=2 ; i++) {
                 String tenantId = "tenant" + i;
                 checkBaseColumnCount(tenantId, null, "TENANT_VIEW1", 4);
-                checkBaseColumnCount(tenantId, null, "TENANT_VIEW2", DIVORCED_VIEW_BASE_COLUMN_COUNT);
-                checkBaseColumnCount(tenantId, null, "TENANT_VIEW3", DIVORCED_VIEW_BASE_COLUMN_COUNT);
+                checkBaseColumnCount(tenantId, null, "TENANT_VIEW2", DIVERGED_VIEW_BASE_COLUMN_COUNT);
+                checkBaseColumnCount(tenantId, null, "TENANT_VIEW3", DIVERGED_VIEW_BASE_COLUMN_COUNT);
             }
             
             // Verify base column count for global views
             checkBaseColumnCount(null, null, "GLOBAL_VIEW1", 4);
-            checkBaseColumnCount(null, null, "GLOBAL_VIEW2", DIVORCED_VIEW_BASE_COLUMN_COUNT);
-            checkBaseColumnCount(null, null, "GLOBAL_VIEW3", DIVORCED_VIEW_BASE_COLUMN_COUNT);
+            checkBaseColumnCount(null, null, "GLOBAL_VIEW2", DIVERGED_VIEW_BASE_COLUMN_COUNT);
+            checkBaseColumnCount(null, null, "GLOBAL_VIEW3", DIVERGED_VIEW_BASE_COLUMN_COUNT);
         }
         
         
@@ -243,7 +243,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
                     conn3.createStatement().execute(
                         "ALTER VIEW " + fullViewName + " DROP COLUMN CF2.V2");
                 }
-                expectedBaseColumnCount = DIVORCED_VIEW_BASE_COLUMN_COUNT;
+                expectedBaseColumnCount = DIVERGED_VIEW_BASE_COLUMN_COUNT;
             } else {
                 expectedBaseColumnCount = 6;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c069bd4/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 43dc07a..0ddd58d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -62,7 +62,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES;
-import static org.apache.phoenix.query.QueryConstants.DIVORCED_VIEW_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
@@ -1588,7 +1588,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             // lock the rows corresponding to views so that no other thread can modify the view meta-data
             RowLock viewRowLock = acquireLock(region, viewKey, locks);
             PTable view = doGetTable(viewKey, clientTimeStamp, viewRowLock);
-            if (view.getBaseColumnCount() == QueryConstants.DIVORCED_VIEW_BASE_COLUMN_COUNT) {
+            if (view.getBaseColumnCount() == QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT) {
                 // if a view has divorced itself from the base table, we don't allow schema changes
                 // to be propagated to it.
                 return;
@@ -1641,7 +1641,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         PDataType.fromTypeId(dataColumnDataType)).getSqlType();
                                 byte[] indexColumnDataTypeBytes = new byte[PInteger.INSTANCE.getByteSize()];
                                 PInteger.INSTANCE.getCodec().encodeInt(indexColumnDataType, indexColumnDataTypeBytes, 0);
-                                indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.DATA_TYPE_BYTES, indexColumnDataTypeBytes);
                             }
                             
@@ -1651,7 +1651,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES);
                             if (decimalDigits != null && decimalDigits.size() > 0) {
                                 Cell decimalDigit = decimalDigits.get(0);
-                                indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES, decimalDigit.getValueArray());
                             }
                             
@@ -1661,7 +1661,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES);
                             if (columnSizes != null && columnSizes.size() > 0) {
                                 Cell columnSize = columnSizes.get(0);
-                                indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES, columnSize.getValueArray());
                             }
                             
@@ -1670,7 +1670,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     PhoenixDatabaseMetaData.SORT_ORDER_BYTES);
                             if (sortOrders != null && sortOrders.size() > 0) {
                                 Cell sortOrder = sortOrders.get(0);
-                                indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.SORT_ORDER_BYTES, sortOrder.getValueArray());
                             }
                             
@@ -1679,7 +1679,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES);
                             if (dataTableNames != null && dataTableNames.size() > 0) {
                                 Cell dataTableName = dataTableNames.get(0);
-                                indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES, dataTableName.getValueArray());
                             }
                             
@@ -1687,12 +1687,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             byte[] ordinalPositionBytes = new byte[PInteger.INSTANCE.getByteSize()];
                             int ordinalPositionOfNewCol = oldNumberOfColsInIndex + deltaNumPkColsSoFar;
                             PInteger.INSTANCE.getCodec().encodeInt(ordinalPositionOfNewCol, ordinalPositionBytes, 0);
-                            indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                            indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, ordinalPositionBytes);
                             
                             // New PK columns have to be nullable after the first DDL
                             byte[] isNullableBytes = PBoolean.INSTANCE.toBytes(true);
-                            indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                            indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                         PhoenixDatabaseMetaData.NULLABLE_BYTES, isNullableBytes);
                             
                             // Set the key sequence for the pk column to be added
@@ -1700,7 +1700,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             short newKeySeq = (short)(currentKeySeq + deltaNumPkColsSoFar);
                             byte[] keySeqBytes = new byte[PSmallint.INSTANCE.getByteSize()];
                             PSmallint.INSTANCE.getCodec().encodeShort(newKeySeq, keySeqBytes, 0);
-                            indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                            indexColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                     PhoenixDatabaseMetaData.KEY_SEQ_BYTES, keySeqBytes);
                             
                             mutationsForAddingColumnsToViews.add(indexColumnDefinitionPut);
@@ -1717,14 +1717,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     long newSequenceNumber = index.getSequenceNumber() + 1;
                     byte[] newSequenceNumberPtr = new byte[PLong.INSTANCE.getByteSize()];
                     PLong.INSTANCE.getCodec().encodeLong(newSequenceNumber, newSequenceNumberPtr, 0);
-                    indexHeaderRowMutation.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                    indexHeaderRowMutation.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                             PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES, newSequenceNumberPtr);
                     
                     // increase the column count
                     int newColumnCount = index.getColumns().size() + deltaNumPkColsSoFar;
                     byte[] newColumnCountPtr = new byte[PInteger.INSTANCE.getByteSize()];
                     PInteger.INSTANCE.getCodec().encodeInt(newColumnCount, newColumnCountPtr, 0);
-                    indexHeaderRowMutation.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                    indexHeaderRowMutation.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                             PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES, newColumnCountPtr);
                     
                     // add index row header key to the invalidate list to force clients to fetch the latest meta-data
@@ -1794,21 +1794,29 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
                     byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
                     PTableType type = table.getType();
-                    TableViewFinderResult childViewsResult = findChildViews(region, tenantId, table,
-                            (type == PTableType.VIEW ? PARENT_TABLE_BYTES : PHYSICAL_TABLE_BYTES));
                     List<Mutation> mutationsForAddingColumnsToViews = Collections.emptyList();
-                    if (childViewsResult.hasViews()) {
-                        /*
-                         * Adding a column is not allowed if: 1) Meta-data for child view/s spans over more than one
-                         * region. 2) Adding column to a views that has child view/s.
-                         */
-                        if (!childViewsResult.allViewsInSingleRegion() || type == PTableType.VIEW) {
-                            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                    EnvironmentEdgeManager.currentTimeMillis(), null);
-                        } else {
-                            mutationsForAddingColumnsToViews = new ArrayList<>(childViewsResult.getResults().size() * tableMetaData.size());
-                            addRowsToChildViews(tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp,
-                                    childViewsResult, region, locks);
+                    /*
+                     * If adding a column to a view, we don't want to propagate those meta-data changes to the child
+                     * view hierarchy. This is because our check of finding child views is expensive and we want making
+                     * meta-data changes to views to be light-weight. The side-effect of this change is that a child
+                     * won't have it's parent views columns i.e. it would have diverged itself from the parent view. See
+                     * https://issues.apache.org/jira/browse/PHOENIX-2051 for a proper way to fix the performance issue
+                     * and https://issues.apache.org/jira/browse/PHOENIX-2054 for enabling meta-data changes to a view
+                     * to be propagated to its view hierarchy.
+                     */
+                    if (type == PTableType.TABLE) {
+                        TableViewFinderResult childViewsResult = findChildViews(region, tenantId, table, PHYSICAL_TABLE_BYTES);
+                        if (childViewsResult.hasViews()) {
+                            // Adding a column is not allowed if the meta-data for child view/s spans over
+                            // more than one region (since the changes cannot be done in a transactional fashion)
+                            if (!childViewsResult.allViewsInSingleRegion()) {
+                                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+                                        EnvironmentEdgeManager.currentTimeMillis(), null);
+                            } else {
+                                mutationsForAddingColumnsToViews = new ArrayList<>(childViewsResult.getResults().size() * tableMetaData.size());
+                                addRowsToChildViews(tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp,
+                                        childViewsResult, region, locks);
+                            }
                         }
                     }
                     for (Mutation m : tableMetaData) {
@@ -2041,18 +2049,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         continue;
                                     }
                                     if (table.getType() == PTableType.VIEW) {
-                                        if (table.getBaseColumnCount() != DIVORCED_VIEW_BASE_COLUMN_COUNT
+                                        if (table.getBaseColumnCount() != DIVERGED_VIEW_BASE_COLUMN_COUNT
                                                 && columnToDelete.getPosition() < table.getBaseColumnCount()) {
                                             /*
                                              * If the column being dropped is inherited from the base table, then the
-                                             * view is about to divorce itself from the base table. Divorce here means
-                                             * that any further meta-data changes made to the base table will not be
-                                             * propagated to the hierarchy of views on the base table.
+                                             * view is about to diverge itself from the base table. The consequence of
+                                             * this divergence is that that any further meta-data changes made to the
+                                             * base table will not be propagated to the hierarchy of views where this
+                                             * view is the root.
                                              */
                                             byte[] viewKey = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
                                             Put updateBaseColumnCountPut = new Put(viewKey);
                                             byte[] baseColumnCountPtr = new byte[PInteger.INSTANCE.getByteSize()];
-                                            PInteger.INSTANCE.getCodec().encodeInt(DIVORCED_VIEW_BASE_COLUMN_COUNT,
+                                            PInteger.INSTANCE.getCodec().encodeInt(DIVERGED_VIEW_BASE_COLUMN_COUNT,
                                                     baseColumnCountPtr, 0);
                                             updateBaseColumnCountPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                                                     PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES, clientTimeStamp,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c069bd4/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index f82c594..d095049 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -185,7 +185,7 @@ public interface QueryConstants {
     public static final BigDecimal BD_MILLIS_IN_DAY = BigDecimal.valueOf(QueryConstants.MILLIS_IN_DAY);
     public static final int MAX_ALLOWED_NANOS = 999999999;
     public static final int NANOS_IN_SECOND = BigDecimal.valueOf(Math.pow(10, 9)).intValue();
-    public static final int DIVORCED_VIEW_BASE_COLUMN_COUNT = -100;
+    public static final int DIVERGED_VIEW_BASE_COLUMN_COUNT = -100;
     public static final int BASE_TABLE_BASE_COLUMN_COUNT = -1;
     public static final String CREATE_TABLE_METADATA =
             // Do not use IF NOT EXISTS as we sometimes catch the TableAlreadyExists

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c069bd4/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index dff6598..8d574ce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -32,7 +32,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.query.QueryConstants.DIVORCED_VIEW_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 
 import java.io.IOException;
 import java.sql.PreparedStatement;
@@ -595,7 +595,7 @@ public class UpgradeUtil {
                         // We are about to iterate through columns of a different view. Check whether base column count was upserted.
                         // If it wasn't then it is likely the case that a column inherited from the base table was dropped from view.
                         if (currentViewName != null && !baseColumnCountUpserted && numBaseTableColsMatched < numColsInBaseTable) {
-                            upsertBaseColumnCountInHeaderRow(metaConnection, currentTenantId, currentViewSchema, currentViewName, DIVORCED_VIEW_BASE_COLUMN_COUNT);
+                            upsertBaseColumnCountInHeaderRow(metaConnection, currentTenantId, currentViewSchema, currentViewName, DIVERGED_VIEW_BASE_COLUMN_COUNT);
                         }
                         // reset the values as we are now going to iterate over columns of a new view.
                         numBaseTableColsMatched = 0;
@@ -641,7 +641,7 @@ public class UpgradeUtil {
                             }
                         } else {
                             // special value to denote that the view has divorced itself from the base physical table.
-                            upsertBaseColumnCountInHeaderRow(metaConnection, viewTenantId, viewSchema, viewName, DIVORCED_VIEW_BASE_COLUMN_COUNT);
+                            upsertBaseColumnCountInHeaderRow(metaConnection, viewTenantId, viewSchema, viewName, DIVERGED_VIEW_BASE_COLUMN_COUNT);
                             baseColumnCountUpserted = true;
                             // ignore rest of the rows for the view.
                             ignore = true;