You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2022/02/24 16:36:58 UTC

[phoenix] 02/02: PHOENIX-6646 System tables are not upgraded after namespace migration

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

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

commit a33c2d02b7dbb042ee719a146ed23c856d730369
Author: Istvan Toth <st...@apache.org>
AuthorDate: Wed Feb 16 14:43:50 2022 +0100

    PHOENIX-6646 System tables are not upgraded after namespace migration
    
    refactor upgrade code to better handle NS mapping + version update case
    fix ConnectionQueryServicesTestImpl to avoid deadlock on upsert...select
    improve handling of missing Hbase system tables on upgrade
    add template for IT that loads HBase table snapshots from local FS
---
 .../end2end/ConnectionQueryServicesTestImpl.java   |  12 ++-
 .../phoenix/end2end/SnapshotTestTemplateIT.java    | 119 +++++++++++++++++++++
 .../org/apache/phoenix/jdbc/PhoenixStatement.java  |   2 +-
 .../phoenix/query/ConnectionQueryServicesImpl.java |  99 ++++++++---------
 .../java/org/apache/phoenix/util/MetaDataUtil.java |  26 +++--
 .../java/org/apache/phoenix/util/UpgradeUtil.java  |  13 +--
 .../java/org/apache/phoenix/query/BaseTest.java    |   2 +-
 7 files changed, 197 insertions(+), 76 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
index c6b0e3f..7e7b6b1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
@@ -23,8 +23,10 @@ import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
@@ -53,9 +55,9 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Sets;
 public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl {
     private static final Logger LOGGER =
             LoggerFactory.getLogger(ConnectionQueryServicesTestImpl.class);
-    protected int NUM_SLAVES_BASE = 1; // number of slaves for the cluster
     // Track open connections to free them on close as unit tests don't always do this.
-    private Set<PhoenixConnection> connections = Sets.newHashSet();
+    private Set<PhoenixConnection> connections =
+            Collections.newSetFromMap(new ConcurrentHashMap<PhoenixConnection, Boolean>());
     private final PhoenixTransactionService[] txServices = new PhoenixTransactionService[TransactionFactory.Provider.values().length];
     
     public ConnectionQueryServicesTestImpl(QueryServices services, ConnectionInfo info, Properties props) throws SQLException {
@@ -63,13 +65,15 @@ public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl
     }
     
     @Override
-    public synchronized void addConnection(PhoenixConnection connection) throws SQLException {
+    public void addConnection(PhoenixConnection connection) throws SQLException {
         connections.add(connection);
+        super.addConnection(connection);
     }
     
     @Override
-    public synchronized void removeConnection(PhoenixConnection connection) throws SQLException {
+    public void removeConnection(PhoenixConnection connection) throws SQLException {
         connections.remove(connection);
+        super.removeConnection(connection);
     }
 
     @Override
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SnapshotTestTemplateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SnapshotTestTemplateIT.java
new file mode 100644
index 0000000..a990ab3
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SnapshotTestTemplateIT.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is a not a standard IT.
+ * It is starting point for writing ITs that load specific tables from a snapshot.
+ * Tests based on this IT are meant for debugging specific problems where HBase table snapshots are
+ * available for replication, and are not meant to be part of the standard test suite
+ * (or even being committed to the ASF branches)
+ */
+
+@Ignore
+@Category(NeedsOwnMiniClusterTest.class)
+public class SnapshotTestTemplateIT extends BaseTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(
+        SnapshotTestTemplateIT.class);
+
+    private static final HashMap<String, String> SNAPSHOTS_TO_LOAD;
+
+    static {
+        SNAPSHOTS_TO_LOAD = new HashMap<>();
+        //Add any HBase tables, including Phoenix System tables
+        SNAPSHOTS_TO_LOAD.put("SYSTEM.CATALOG_SNAPSHOT", "/path/to/local/snapshot_dir");
+    }
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+
+        //Start minicluster without Phoenix first
+        checkClusterInitialized(new ReadOnlyProps(serverProps.entrySet().iterator()));
+
+        //load snapshots int HBase
+        for (Entry<String, String> snapshot : SNAPSHOTS_TO_LOAD.entrySet()) {
+            importSnapshot(snapshot.getKey(), snapshot.getValue());
+        }
+
+        //Now we can start Phoenix
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet()
+                .iterator()));
+    }
+
+    private static void importSnapshot(String key, String value) throws IOException {
+        LOGGER.info("importing {} snapshot from {}", key, value);
+        // copy local snapshot dir to Minicluster HDFS
+        Path localPath = new Path(value);
+        assertTrue(FileSystem.getLocal(config).exists(new Path(localPath, ".hbase-snapshot")));
+        FileSystem hdfsFs = FileSystem.get(config);
+        Path hdfsImportPath = new Path(hdfsFs.getHomeDirectory(), "snapshot-import" + "/" + key + "/");
+        assertTrue(hdfsFs.mkdirs(hdfsImportPath));
+        hdfsFs.copyFromLocalFile(localPath, hdfsImportPath);
+        hdfsImportPath = new Path(hdfsImportPath, localPath.getName());
+        assertTrue(hdfsFs.exists(new Path(hdfsImportPath, ".hbase-snapshot")));
+
+        //import the snapshot
+        ExportSnapshot exportTool = new ExportSnapshot();
+        exportTool.setConf(config);
+        int importExitCode = exportTool.run(new String[] {
+                "-snapshot", key,
+                "-copy-from", hdfsImportPath.toUri().toString(),
+                "-copy-to", CommonFSUtils.getRootDir(config).toUri().toString()
+                });
+        assertEquals(0, importExitCode);
+
+        //load the snapshot
+        utility.getHBaseAdmin().restoreSnapshot(key);
+    }
+
+    @Test
+    public void testDummy() throws Exception {
+        assertTrue(true);
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 10c379b..e4eb688 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -388,7 +388,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     }
 
     private int executeMutation(final CompilableStatement stmt, final boolean doRetryOnMetaNotFoundError) throws SQLException {
-	 if (connection.isReadOnly()) {
+        if (connection.isReadOnly()) {
             throw new SQLExceptionInfo.Builder(
                 SQLExceptionCode.READ_ONLY_CONNECTION).
                 build().buildException();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 93a2a5f..1a79a7d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -149,6 +149,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -3755,8 +3756,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
             addViewIndexToParentLinks(metaConnection);
-        }
-        if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
             metaConnection = addColumnsIfNotExists(
                     metaConnection,
                     PhoenixDatabaseMetaData.SYSTEM_CATALOG,
@@ -3851,13 +3850,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
             metaConnection = new PhoenixConnection(ConnectionQueryServicesImpl.this, globalUrl,
-                    scnProps, newEmptyMetaData());
+                    scnProps, latestMetaData);
             metaConnection.setRunningUpgrade(true);
             // Always try to create SYSTEM.MUTEX table first since we need it to acquire the
             // upgrade mutex. Upgrade or migration is not possible without the upgrade mutex
             try (HBaseAdmin admin = getAdmin()) {
                 createSysMutexTableIfNotExists(admin);
             }
+            UpgradeRequiredException caughtUpgradeRequiredException = null;
+            TableAlreadyExistsException caughtTableAlreadyExistsException = null;
             try {
                 metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
             } catch (NewerTableAlreadyExistsException ignore) {
@@ -3867,61 +3868,43 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } catch (UpgradeRequiredException e) {
                 // This is thrown while trying to create SYSTEM:CATALOG to indicate that we must
                 // migrate SYSTEM tables to the SYSTEM namespace and/or upgrade SYSCAT if required
-                long currentServerSideTableTimeStamp = e.getSystemCatalogTimeStamp();
-                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
-                    moveChildLinks = true;
-                }
-                sysCatalogTableName = SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getNameAsString();
-                if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, ConnectionQueryServicesImpl.this.getProps())) {
-                    String snapshotName = null;
-                    // Try acquiring a lock in SYSMUTEX table before migrating the tables since it involves disabling the table.
-                    if (acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP)) {
-                        LOGGER.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace "
-                          + "and/or upgrading " + sysCatalogTableName);
-                        snapshotName = getSysTableSnapshotName(
-                            currentServerSideTableTimeStamp, SYSTEM_CATALOG_NAME);
-                        createSnapshot(snapshotName, SYSTEM_CATALOG_NAME);
-                        systemTableToSnapshotMap.put(SYSTEM_CATALOG_NAME,
-                            snapshotName);
-                        LOGGER.info("Created snapshot {} for {}", snapshotName,
-                            SYSTEM_CATALOG_NAME);
-                    }
-                    // We will not reach here if we fail to acquire the lock, since it throws UpgradeInProgressException
-
-                    // If SYSTEM tables exist, they are migrated to HBase SYSTEM namespace
-                    // If they don't exist or they're already migrated, this method will return immediately
-                    ensureSystemTablesMigratedToSystemNamespace();
-                    LOGGER.debug("Migrated SYSTEM tables to SYSTEM namespace");
-                    if (snapshotName != null) {
-                        deleteSnapshot(snapshotName);
-                    } else {
-                        snapshotName = getSysTableSnapshotName(
-                            currentServerSideTableTimeStamp, SYSTEM_CATALOG_NAME);
-                    }
-                    systemTableToSnapshotMap.remove(SYSTEM_CATALOG_NAME);
-                    // take snapshot of SYSTEM:CATALOG
-                    createSnapshot(snapshotName, sysCatalogTableName);
-                    systemTableToSnapshotMap.put(sysCatalogTableName,
-                        snapshotName);
-                    LOGGER.info("Created snapshot {} for {}", snapshotName,
-                        sysCatalogTableName);
-
-                    metaConnection = upgradeSystemCatalogIfRequired(metaConnection,
-                            currentServerSideTableTimeStamp);
-                }
+                caughtUpgradeRequiredException = e;
             } catch (TableAlreadyExistsException e) {
-                long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
-                sysCatalogTableName = e.getTable().getPhysicalName().getString();
-                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
-                    // Try acquiring a lock in SYSMUTEX table before upgrading SYSCAT. If we cannot acquire the lock,
-                    // it means some old client is either migrating SYSTEM tables or trying to upgrade the schema of
-                    // SYSCAT table and hence it should not be interrupted
-                    if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp)) {
-                        LOGGER.debug("Acquired lock in SYSMUTEX table for upgrading " + sysCatalogTableName);
-                        takeSnapshotOfSysTable(systemTableToSnapshotMap, e);
+                caughtTableAlreadyExistsException = e;
+            }
+
+            if (caughtUpgradeRequiredException != null
+                    || caughtTableAlreadyExistsException != null) {
+                long currentServerSideTableTimeStamp;
+                if (caughtUpgradeRequiredException != null) {
+                    currentServerSideTableTimeStamp =
+                            caughtUpgradeRequiredException.getSystemCatalogTimeStamp();
+                } else {
+                    currentServerSideTableTimeStamp =
+                            caughtTableAlreadyExistsException.getTable().getTimeStamp();
+                }
+                acquiredMutexLock = acquireUpgradeMutex(
+                    MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP);
+                LOGGER.debug(
+                    "Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM "
+                    + "namespace and/or upgrading " + sysCatalogTableName);
+                String snapshotName = getSysTableSnapshotName(currentServerSideTableTimeStamp,
+                    SYSTEM_CATALOG_NAME);
+                createSnapshot(snapshotName, SYSTEM_CATALOG_NAME);
+                systemTableToSnapshotMap.put(SYSTEM_CATALOG_NAME, snapshotName);
+                LOGGER.info("Created snapshot {} for {}", snapshotName, SYSTEM_CATALOG_NAME);
+
+                if (caughtUpgradeRequiredException != null) {
+                    if (SchemaUtil.isNamespaceMappingEnabled(
+                            PTableType.SYSTEM, ConnectionQueryServicesImpl.this.getProps())) {
+                        // If SYSTEM tables exist, they are migrated to HBase SYSTEM namespace
+                        // If they don't exist or they're already migrated, this method will return
+                        //immediately
+                        ensureSystemTablesMigratedToSystemNamespace();
+                        LOGGER.debug("Migrated SYSTEM tables to SYSTEM namespace");
                     }
-                    // We will not reach here if we fail to acquire the lock, since it throws UpgradeInProgressException
                 }
+
                 metaConnection = upgradeSystemCatalogIfRequired(metaConnection, currentServerSideTableTimeStamp);
                 if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
                     moveChildLinks = true;
@@ -4408,6 +4391,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             admin.snapshot(snapshotName, tableName);
             LOGGER.info("Successfully created snapshot " + snapshotName + " for "
                     + tableName);
+        } catch (SnapshotCreationException e) {
+            if (e.getMessage().contains("doesn't exist")) {
+                LOGGER.warn("Could not create snapshot {}, table is missing." + snapshotName, e);
+            } else {
+                sqlE = new SQLException(e);
+            }
         } catch (Exception e) {
             sqlE = new SQLException(e);
         } finally {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 7c4ef6d..96f474d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -242,8 +242,10 @@ public class MetaDataUtil {
         return ((int)((version << Byte.SIZE * 3) >>> Byte.SIZE * 7) & 0x1) != 0;
     }
 
-    // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch.
-    // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch.
+    // The first three bytes of the long encode the HBase version as major.minor.patch.
+    // The fourth byte is isTableNamespaceMappingEnabled
+    // The fifth to seventh bytes of the value encode the Phoenix version as major.minor.patch.
+    // The eights byte encodes whether the WAL codec is correctly installed
     /**
      * Encode HBase and Phoenix version along with some server-side config information such as whether WAL codec is
      * installed (necessary for non transactional, mutable secondar indexing), and whether systemNamespace mapping is enabled.
@@ -262,7 +264,9 @@ public class MetaDataUtil {
         long version =
         // Encode HBase major, minor, patch version
         (hbaseVersion << (Byte.SIZE * 5))
-                // Encode if systemMappingEnabled are enabled on the server side
+                // Encode if table namespace mapping is enabled on the server side
+                // Note that we DO NOT return information on whether system tables are mapped
+                // on the server side
                 | (isTableNamespaceMappingEnabled << (Byte.SIZE * 4))
                 // Encode Phoenix major, minor, patch version
                 | (phoenixVersion << (Byte.SIZE * 1))
@@ -764,7 +768,9 @@ public class MetaDataUtil {
     public static boolean hasLocalIndexTable(PhoenixConnection connection, byte[] physicalTableName) throws SQLException {
         try {
             HTableDescriptor desc = connection.getQueryServices().getTableDescriptor(physicalTableName);
-            if(desc == null ) return false;
+            if (desc == null ) {
+                return false;
+            }
             return hasLocalIndexColumnFamily(desc);
         } catch (TableNotFoundException e) {
             return false;
@@ -792,7 +798,9 @@ public class MetaDataUtil {
 
     public static List<byte[]> getLocalIndexColumnFamilies(PhoenixConnection conn, byte[] physicalTableName) throws SQLException {
         HTableDescriptor desc = conn.getQueryServices().getTableDescriptor(physicalTableName);
-        if(desc == null ) return Collections.emptyList();
+        if (desc == null ) {
+            return Collections.emptyList();
+        }
         List<byte[]> families = new ArrayList<byte[]>(desc.getColumnFamilies().length / 2);
         for (HColumnDescriptor cf : desc.getColumnFamilies()) {
             if (cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
@@ -1036,19 +1044,19 @@ public class MetaDataUtil {
             connection.setAutoCommit(true);
             Set<String> physicalTablesSet = new HashSet<>();
             physicalTablesSet.add(table.getPhysicalName().getString());
-            for(byte[] physicalTableName:physicalTableNames) {
+            for (byte[] physicalTableName:physicalTableNames) {
                 physicalTablesSet.add(Bytes.toString(physicalTableName));
             }
-            for(MetaDataProtocol.SharedTableState s: sharedTableStates) {
+            for (MetaDataProtocol.SharedTableState s: sharedTableStates) {
                 physicalTablesSet.add(s.getPhysicalNames().get(0).getString());
             }
             StringBuilder buf = new StringBuilder("DELETE FROM SYSTEM.STATS WHERE PHYSICAL_NAME IN (");
             Iterator itr = physicalTablesSet.iterator();
-            while(itr.hasNext()) {
+            while (itr.hasNext()) {
                 buf.append("'" + itr.next() + "',");
             }
             buf.setCharAt(buf.length() - 1, ')');
-            if(table.getIndexType()==IndexType.LOCAL) {
+            if (table.getIndexType()==IndexType.LOCAL) {
                 buf.append(" AND COLUMN_FAMILY IN(");
                 if (table.getColumnFamilies().isEmpty()) {
                     buf.append("'" + QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY + "',");
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 a1feaf4..09c5fa3 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
@@ -369,12 +369,12 @@ public class UpgradeUtil {
                         HTableDescriptor dataTableDesc = admin.getTableDescriptor(TableName.valueOf(MetaDataUtil.getLocalIndexUserTableName(table.getNameAsString())));
                         HColumnDescriptor[] columnFamilies = dataTableDesc.getColumnFamilies();
                         boolean modifyTable = false;
-                        for(HColumnDescriptor cf : columnFamilies) {
+                        for (HColumnDescriptor cf : columnFamilies) {
                             String localIndexCf = QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX+cf.getNameAsString();
-                            if(dataTableDesc.getFamily(Bytes.toBytes(localIndexCf))==null){
+                            if (dataTableDesc.getFamily(Bytes.toBytes(localIndexCf))==null){
                                 HColumnDescriptor colDef =
                                         new HColumnDescriptor(localIndexCf);
-                                for(Entry<ImmutableBytesWritable, ImmutableBytesWritable>keyValue: cf.getValues().entrySet()){
+                                for (Entry<ImmutableBytesWritable, ImmutableBytesWritable>keyValue: cf.getValues().entrySet()){
                                     colDef.setValue(keyValue.getKey().copyBytes(), keyValue.getValue().copyBytes());
                                 }
                                 dataTableDesc.addFamily(colDef);
@@ -382,8 +382,8 @@ public class UpgradeUtil {
                             }
                         }
                         List<String> coprocessors = dataTableDesc.getCoprocessors();
-                        for(String coprocessor:  coprocessors) {
-                            if(coprocessor.equals(localIndexSplitter)) {
+                        for (String coprocessor:  coprocessors) {
+                            if (coprocessor.equals(localIndexSplitter)) {
                                 dataTableDesc.removeCoprocessor(localIndexSplitter);
                                 modifyTable = true;
                             }
@@ -621,7 +621,7 @@ public class UpgradeUtil {
                         String indexPhysicalTableName = MetaDataUtil.getViewIndexPhysicalName(viewPTable.getPhysicalName().getString());
                         if (physicalTables.add(indexPhysicalTableName)) {
                             final TableName tableName = TableName.valueOf(indexPhysicalTableName);
-                            if(admin.tableExists(tableName)) {
+                            if (admin.tableExists(tableName)) {
                                 admin.disableTable(tableName);
                                 admin.truncateTable(tableName, false);
                             }
@@ -2641,4 +2641,5 @@ public class UpgradeUtil {
         return tableDesc.getFamily(
             SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions() > 1;
     }
+
 }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index a5a85f0..a8e4c64 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -430,7 +430,7 @@ public abstract class BaseTest {
         return url;
     }
     
-    private static String checkClusterInitialized(ReadOnlyProps serverProps) throws Exception {
+    protected static String checkClusterInitialized(ReadOnlyProps serverProps) throws Exception {
         if (!clusterInitialized) {
             url = setUpTestCluster(config, serverProps);
             clusterInitialized = true;