You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/12/27 22:15:40 UTC

[1/2] phoenix git commit: PHOENIX-4487 Missing SYSTEM.MUTEX table upgrading from 4.7 to 4.13

Repository: phoenix
Updated Branches:
  refs/heads/master 34693843a -> 2759727e4


PHOENIX-4487 Missing SYSTEM.MUTEX table upgrading from 4.7 to 4.13


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

Branch: refs/heads/master
Commit: 2759727e444ef8ae7475d25979be84bfe89895f5
Parents: 1636f61
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Dec 22 10:58:55 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Dec 26 13:21:09 2017 -0800

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 29 ++++++++++++++++----
 1 file changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2759727e/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
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 072bf28..38be6af 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
@@ -2640,11 +2640,23 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } catch (TableAlreadyExistsException e) {
                 long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                 sysCatalogTableName = e.getTable().getPhysicalName().getString();
-                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP
-                        && (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey))) {
-                    snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
-                    createSnapshot(snapshotName, sysCatalogTableName);
-                    snapshotCreated = true;
+                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
+                    // Ensure that the SYSTEM.MUTEX table has been created prior
+                    // to attempting to acquire the upgrade mutex. If namespace
+                    // mapping is enabled, we've already done this earlier in the
+                    // upgrade, so no need for a bunch of wasted RPCs.
+                    if (currentServerSideTableTimeStamp <= MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 &&
+                            !SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
+                                    ConnectionQueryServicesImpl.this.getProps())) {
+                        try (HBaseAdmin admin = getAdmin()) {
+                            createSysMutexTable(admin, this.getProps());
+                        }
+                    }
+                    if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey)) {
+                        snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
+                        createSnapshot(snapshotName, sysCatalogTableName);
+                        snapshotCreated = true;
+                    }
                 }
                 String columnsToAdd = "";
                 // This will occur if we have an older SYSTEM.CATALOG and we need to update it to
@@ -3191,6 +3203,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // Try acquiring a lock in SYSMUTEX table before migrating the tables since it involves disabling the table
             // If we cannot acquire lock, it means some old client is either migrating SYSCAT or trying to upgrade the
             // schema of SYSCAT table and hence it should not be interrupted
+            // Create mutex if not already created
+            if (!tableNames.contains(PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME)) {
+                TableName mutexName = SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
+                if (PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME.equals(mutexName) || !tableNames.contains(mutexName)) {
+                    createSysMutexTable(admin, props);
+                }
+            }
             acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey);
             if(acquiredMutexLock) {
                 logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace");


[2/2] phoenix git commit: PHOENIX-4488 Cache config parameters for MetaDataEndPointImpl during initialization

Posted by ja...@apache.org.
PHOENIX-4488 Cache config parameters for MetaDataEndPointImpl during initialization


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

Branch: refs/heads/master
Commit: 1636f6182ae89e9c4b97c877aa919a6edac5bbc2
Parents: 3469384
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Dec 22 11:36:44 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Dec 26 13:21:09 2017 -0800

----------------------------------------------------------------------
 .../coprocessor/MetaDataEndpointImplTest.java   | 44 --------------------
 .../coprocessor/MetaDataEndpointImpl.java       | 30 ++++++-------
 2 files changed, 16 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1636f618/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
deleted file mode 100644
index 2c558d8..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.coprocessor;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class MetaDataEndpointImplTest {
-
-    @Test
-    public void testExceededIndexQuota() throws Exception {
-        PTable parentTable = mock(PTable.class);
-        List<PTable> indexes = Lists.newArrayList(mock(PTable.class), mock(PTable.class));
-        when(parentTable.getIndexes()).thenReturn(indexes);
-        Configuration configuration = new Configuration();
-        assertFalse(MetaDataEndpointImpl.execeededIndexQuota(PTableType.INDEX, parentTable, configuration));
-        configuration.setInt(QueryServices.MAX_INDEXES_PER_TABLE, 1);
-        assertTrue(MetaDataEndpointImpl.execeededIndexQuota(PTableType.INDEX, parentTable, configuration));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1636f618/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 afbd63f..7ad9028 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
@@ -242,7 +242,6 @@ import org.apache.phoenix.util.UpgradeUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.Cache;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -472,6 +471,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private PhoenixMetaDataCoprocessorHost phoenixAccessCoprocessorHost;
     private boolean accessCheckEnabled;
+    private boolean blockWriteRebuildIndex;
+    private int maxIndexesPerTable;
+    private boolean isTablesMappingEnabled;
+
 
     /**
      * Stores a reference to the coprocessor environment provided by the
@@ -492,8 +495,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         
         phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
-        this.accessCheckEnabled = env.getConfiguration().getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+        Configuration config = env.getConfiguration();
+        this.accessCheckEnabled = config.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
                 QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
+        this.blockWriteRebuildIndex  = config.getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
+                QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
+        this.maxIndexesPerTable = config.getInt(QueryServices.MAX_INDEXES_PER_TABLE,
+                    QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+        this.isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                new ReadOnlyProps(config.iterator()));
+
         logger.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
         Tracing.addTraceMetricsSource();
@@ -583,8 +594,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             PTable oldTable = (PTable)metaDataCache.getIfPresent(cacheKey);
             long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
             PTable newTable;
-            boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
-                    QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
             newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion);
             if (newTable == null) {
                 return null;
@@ -1551,7 +1560,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             return;
                         }
                         // make sure we haven't gone over our threshold for indexes on this table.
-                        if (execeededIndexQuota(tableType, parentTable, env.getConfiguration())) {
+                        if (execeededIndexQuota(tableType, parentTable)) {
                             builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
                             builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                             done.run(builder.build());
@@ -1758,11 +1767,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    @VisibleForTesting
-    static boolean execeededIndexQuota(PTableType tableType, PTable parentTable, Configuration configuration) {
-        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= configuration
-            .getInt(QueryServices.MAX_INDEXES_PER_TABLE,
-                QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+    private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
+        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
     }
 
     private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
@@ -3265,8 +3271,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * from getting rebuilt too often.
          */
         final boolean wasLocked = (rowLock != null);
-        boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
-                QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         if (!wasLocked) {
             rowLock = acquireLock(region, key, null);
         }
@@ -3558,8 +3562,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         GetVersionResponse.Builder builder = GetVersionResponse.newBuilder();
         Configuration config = env.getConfiguration();
-        boolean isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
-                new ReadOnlyProps(config.iterator()));
         if (isTablesMappingEnabled
                 && PhoenixDatabaseMetaData.MIN_NAMESPACE_MAPPED_PHOENIX_VERSION > request.getClientVersion()) {
             logger.error("Old client is not compatible when" + " system tables are upgraded to map to namespace");