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 2016/10/27 16:13:31 UTC

phoenix git commit: PHOENIX-3407 Read only RowLock may lead to corrupting SYSTEM.CATALOG and non atomic sequences in HBase 1.2

Repository: phoenix
Updated Branches:
  refs/heads/master 53ca28827 -> 76efe0079


PHOENIX-3407 Read only RowLock may lead to corrupting SYSTEM.CATALOG and non atomic sequences in HBase 1.2


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

Branch: refs/heads/master
Commit: 76efe0079d856b329022834bc8f9c5808337dbd8
Parents: 53ca288
Author: James Taylor <ja...@apache.org>
Authored: Thu Oct 27 09:13:18 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu Oct 27 09:13:18 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/coprocessor/MetaDataEndpointImpl.java    | 9 ++++-----
 .../apache/phoenix/coprocessor/SequenceRegionObserver.java  | 2 +-
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/76efe007/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 85b6c57..1300301 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
@@ -150,7 +150,6 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
-import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
@@ -1586,7 +1585,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private static RowLock acquireLock(Region region, byte[] key, List<RowLock> locks)
         throws IOException {
-        RowLock rowLock = region.getRowLock(key, true);
+        RowLock rowLock = region.getRowLock(key, false);
         if (rowLock == null) {
             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
         }
@@ -2906,7 +2905,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
                 QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         if (!wasLocked) {
-            rowLock = region.getRowLock(key, true);
+            rowLock = region.getRowLock(key, false);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }
@@ -2970,7 +2969,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         try {
             rowLocks = new ArrayList<Region.RowLock>(keys.size());
             for (int i = 0; i < keys.size(); i++) {
-                Region.RowLock rowLock = region.getRowLock(keys.get(i), true);
+                Region.RowLock rowLock = region.getRowLock(keys.get(i), false);
                 if (rowLock == null) {
                     throw new IOException("Failed to acquire lock on "
                             + Bytes.toStringBinary(keys.get(i)));
@@ -3261,7 +3260,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             PIndexState newState =
                     PIndexState.fromSerializedValue(newKV.getValueArray()[newKV.getValueOffset()]);
-            RowLock rowLock = region.getRowLock(key, true);
+            RowLock rowLock = region.getRowLock(key, false);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/76efe007/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
index 1b44c72..6773f36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -91,7 +91,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
     
     private static void acquireLock(Region region, byte[] key, List<RowLock> locks)
         throws IOException {
-        RowLock rowLock = region.getRowLock(key, true);
+        RowLock rowLock = region.getRowLock(key, false);
         if (rowLock == null) {
             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
         }