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:23:03 UTC

phoenix git commit: PHOENIX-4488 Cache config parameters for MetaDataEndPointImpl during initialization

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 5d413a677 -> 8a00ab45b


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/8a00ab45
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/8a00ab45
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/8a00ab45

Branch: refs/heads/4.x-HBase-0.98
Commit: 8a00ab45b09d63a1e198c203d866f88f3b00695d
Parents: 5d413a6
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Dec 22 11:36:44 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Dec 27 14:21:52 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a00ab45/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/8a00ab45/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 3b8cdf7..15d5bdc 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
@@ -235,7 +235,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;
@@ -462,6 +461,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private RegionCoprocessorEnvironment env;
 
+    private boolean blockWriteRebuildIndex;
+    private int maxIndexesPerTable;
+    private boolean isTablesMappingEnabled;
+
+
     /**
      * Stores a reference to the coprocessor environment provided by the
      * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this
@@ -479,6 +483,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
+        
+        Configuration config = env.getConfiguration();
+        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();
@@ -561,8 +574,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;
@@ -1415,7 +1426,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());
@@ -1622,11 +1633,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 RowLock acquireLock(HRegion region, byte[] key, List<RowLock> locks)
@@ -3118,8 +3126,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 = region.getRowLock(key);
             if (rowLock == null) {
@@ -3418,8 +3424,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");