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/12/11 03:43:48 UTC

[10/52] [abbrv] phoenix git commit: PHOENIX-2450 Cleanup API for determining if non transactional mutable secondary index configured properly

PHOENIX-2450 Cleanup API for determining if non transactional mutable secondary index configured properly


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

Branch: refs/heads/calcite
Commit: c98de94f8004566d838de53fb7bc6c150c08e837
Parents: cc9929b
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Nov 24 11:32:17 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Nov 24 11:32:17 2015 -0800

----------------------------------------------------------------------
 .../coprocessor/MetaDataEndpointImpl.java       |  2 +-
 .../phoenix/query/ConnectionQueryServices.java  |  2 +-
 .../query/ConnectionQueryServicesImpl.java      | 26 ++++++++++----------
 .../query/ConnectionlessQueryServicesImpl.java  |  2 +-
 .../query/DelegateConnectionQueryServices.java  |  4 +--
 .../apache/phoenix/schema/MetaDataClient.java   |  8 +++---
 .../org/apache/phoenix/util/MetaDataUtil.java   |  4 +--
 7 files changed, 24 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/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 0a313be..7c80f23 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
@@ -2674,7 +2674,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // The last byte is used to communicate whether or not mutable secondary indexing
         // was configured properly.
         version =
-                MetaDataUtil.encodeMutableIndexConfiguredProperly(version,
+                MetaDataUtil.encodeHasIndexWALCodec(version,
                     IndexManagementUtil.isWALEditCodecSet(this.env.getConfiguration()));
 
         builder.setVersion(version);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index 08c417f..fc41706 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -90,7 +90,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
 
     void clearTableRegionCache(byte[] tableName) throws SQLException;
 
-    boolean isMutableIndexWALCodecInstalled();
+    boolean hasIndexWALCodec();
     
     long createSequence(String tenantId, String schemaName, String sequenceName, long startWith, long incrementBy, long cacheSize, long minValue, long maxValue, boolean cycle, long timestamp) throws SQLException;
     long dropSequence(String tenantId, String schemaName, String sequenceName, long timestamp) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/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 2a08f45..df46013 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
@@ -173,12 +173,6 @@ import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.distributed.PooledClientProvider;
-import co.cask.tephra.distributed.TransactionServiceClient;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
 import com.google.common.cache.Cache;
@@ -188,6 +182,12 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.distributed.PooledClientProvider;
+import co.cask.tephra.distributed.TransactionServiceClient;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 
 public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesImpl.class);
@@ -211,7 +211,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     // Lowest HBase version on the cluster.
     private int lowestClusterHBaseVersion = Integer.MAX_VALUE;
-    private boolean isMutableIndexWALCodecInstalled = true;
+    private boolean hasIndexWALCodec = true;
 
     @GuardedBy("connectionCountLock")
     private int connectionCount = 0;
@@ -1047,11 +1047,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     	}
     }
 
-    private static boolean isInvalidMutableIndexConfig(Long serverVersion) {
+    private static boolean hasIndexWALCodec(Long serverVersion) {
         if (serverVersion == null) {
-            return false;
+            return true;
         }
-        return !MetaDataUtil.decodeMutableIndexConfiguredProperly(serverVersion);
+        return MetaDataUtil.decodeHasIndexWALCodec(serverVersion);
     }
 
     private static boolean isCompatible(Long serverVersion) {
@@ -1103,7 +1103,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     buf.append(name);
                     buf.append(';');
                 }
-                isMutableIndexWALCodecInstalled &= !isInvalidMutableIndexConfig(result.getValue());
+                hasIndexWALCodec &= hasIndexWALCodec(result.getValue());
                 if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(result.getValue())) {
                     minHBaseVersion = MetaDataUtil.decodeHBaseVersion(result.getValue());
                 }
@@ -2485,8 +2485,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public boolean isMutableIndexWALCodecInstalled() {
-        return isMutableIndexWALCodecInstalled;
+    public boolean hasIndexWALCodec() {
+        return hasIndexWALCodec;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 77e4ba9..e983a4c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -390,7 +390,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public boolean isMutableIndexWALCodecInstalled() {
+    public boolean hasIndexWALCodec() {
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 4952355..ca662be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -172,8 +172,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public boolean isMutableIndexWALCodecInstalled() {
-        return getDelegate().isMutableIndexWALCodecInstalled();
+    public boolean hasIndexWALCodec() {
+        return getDelegate().hasIndexWALCodec();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index cb9b831..29e4af2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -200,8 +200,6 @@ import org.apache.phoenix.util.UpgradeUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import co.cask.tephra.TxConstants;
-
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
@@ -210,6 +208,8 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
 
+import co.cask.tephra.TxConstants;
+
 public class MetaDataClient {
     private static final Logger logger = LoggerFactory.getLogger(MetaDataClient.class);
 
@@ -1181,7 +1181,7 @@ public class MetaDataClient {
                     if (hbaseVersion < PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES).setTableName(indexTableName.getTableName()).build().buildException();
                     }
-                    if (!connection.getQueryServices().isMutableIndexWALCodecInstalled() && !dataTable.isTransactional()) {
+                    if (!connection.getQueryServices().hasIndexWALCodec() && !dataTable.isTransactional()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG).setTableName(indexTableName.getTableName()).build().buildException();
                     }
                 }
@@ -2716,7 +2716,7 @@ public class MetaDataClient {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES)
                             .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }
-                        if (connection.getQueryServices().isMutableIndexWALCodecInstalled() && !table.isTransactional()) {
+                        if (!connection.getQueryServices().hasIndexWALCodec() && !table.isTransactional()) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG)
                             .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c98de94f/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
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 47d3bc3..e50513d 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
@@ -107,14 +107,14 @@ public class MetaDataUtil {
     }
     
     // TODO: generalize this to use two bytes to return a SQL error code instead
-    public static long encodeMutableIndexConfiguredProperly(long version, boolean isValid) {
+    public static long encodeHasIndexWALCodec(long version, boolean isValid) {
         if (!isValid) {
             return version | 1;
         }
         return version;
     }
     
-    public static boolean decodeMutableIndexConfiguredProperly(long version) {
+    public static boolean decodeHasIndexWALCodec(long version) {
         return (version & 0xF) == 0;
     }