You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by el...@apache.org on 2018/03/20 22:36:46 UTC

[1/4] phoenix git commit: PHOENIX-4661 Handled deleted PTables in the MetadataCache

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 332caa000 -> c9e46a6fc
  refs/heads/4.x-HBase-1.3 eaa6cdd2d -> a67869eba
  refs/heads/5.x-HBase-2.0 dc7e377d4 -> 7e8abb8a4
  refs/heads/master b6e33f30e -> e9324cc81


PHOENIX-4661 Handled deleted PTables in the MetadataCache

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Sergey Soldatov <ss...@apache.org>


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 7e8abb8a45eed25988848f0cda0f5ce6894043a3
Parents: dc7e377
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Mar 20 14:54:05 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Mar 20 17:32:59 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DropTableIT.java | 43 ++++++++++++++++++++
 .../coprocessor/MetaDataEndpointImpl.java       | 33 ++++++++-------
 .../coprocessor/PhoenixAccessController.java    |  8 +---
 3 files changed, 64 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e8abb8a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
new file mode 100644
index 0000000..823605d
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
@@ -0,0 +1,43 @@
+/*
+ * 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.assertFalse;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+
+import org.junit.Test;
+
+public class DropTableIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testRepeatedDropTable() throws Exception {
+      final String tableName = generateUniqueName();
+      final String url = getUrl();
+      try (final Connection conn = DriverManager.getConnection(url);
+          final Statement stmt = conn.createStatement()) {
+        assertFalse(stmt.execute(String.format("CREATE TABLE %s(pk varchar not null primary key)", tableName)));
+        String dropTable = String.format("DROP TABLE IF EXISTS %s", tableName);
+        for (int i = 0; i < 5; i++) {
+          assertFalse(stmt.execute(dropTable));
+        }
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e8abb8a/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 72906b4..f09f6ee 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
@@ -1282,6 +1282,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
         return function.getFunctionName() == null;
     }
 
+    private PTable getTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
+            long clientTimeStamp, long asOfTimeStamp, int clientVersion) throws IOException, SQLException {
+        PTable table = loadTable(env, key, cacheKey, clientTimeStamp, asOfTimeStamp, clientVersion);
+        if (table == null || isTableDeleted(table)) { return null; }
+        return table;
+    }
+
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
         throws IOException, SQLException {
@@ -1453,7 +1460,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                     
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                    PTable parentTable = getTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
                             clientTimeStamp, clientTimeStamp, clientVersion);
                     if (parentTable == null) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
@@ -1468,13 +1475,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                         byte[] parentKey = SchemaUtil.getTableKey(
                                 parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
                                 parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                        parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                 clientTimeStamp, clientTimeStamp, clientVersion);
                         if (parentTable == null) {
                             // it could be a global view
                             parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                                     parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                            parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                     clientTimeStamp, clientTimeStamp, clientVersion);
                         }
                     }
@@ -1971,12 +1978,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
             byte[] key =
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
-
-            
+            Region region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
             PTableType ptableType=PTableType.fromSerializedValue(tableType);
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
             byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
-            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+            PTable loadedTable = getTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
                     request.getClientVersion());
             if (loadedTable == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
@@ -1988,13 +1999,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                     SchemaUtil.getTableName(schemaName, tableName),
                     TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
                     getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
-
-            Region region = env.getRegion();
-            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
-            if (result != null) {
-                done.run(MetaDataMutationResult.toProto(result));
-                return;
-            }
             List<RowLock> locks = Lists.newArrayList();
 
             try {
@@ -2002,6 +2006,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                 if (key != lockKey) {
                     ServerUtil.acquireLock(region, key, locks);
                 }
+
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName,
@@ -3652,7 +3657,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
 
                 //check permission on data table
                 long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                PTable loadedTable = getTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
                         request.getClientVersion());
                 if (loadedTable == null) {
                     builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e8abb8a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 314d00c..2396c91 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -22,7 +22,6 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
@@ -69,7 +68,6 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 
-import com.google.common.collect.Lists;
 import com.google.protobuf.RpcCallback;
 
 public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
@@ -113,10 +111,8 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     @Override
     public void preGetTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
             String tableName, TableName physicalTableName) throws IOException {
-        for (MasterObserver observer : getAccessControllers()) {
-            observer.preGetTableDescriptors(getMasterObsevrverContext(), Lists.newArrayList(physicalTableName),
-                    Collections.<TableDescriptor> emptyList(), null);
-        }
+        if (!accessCheckEnabled) { return; }
+        requireAccess("GetTable" + tenantId, physicalTableName, Action.READ, Action.EXEC);
     }
 
     @Override


[3/4] phoenix git commit: PHOENIX-4661 Handled deleted PTables in the MetadataCache

Posted by el...@apache.org.
PHOENIX-4661 Handled deleted PTables in the MetadataCache

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Sergey Soldatov <ss...@apache.org>


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

Branch: refs/heads/4.x-HBase-1.3
Commit: a67869eba1c934197c240ed977678275fa734f00
Parents: eaa6cdd
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Mar 20 14:54:05 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Mar 20 18:07:44 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DropTableIT.java | 43 ++++++++++++++++++++
 .../coprocessor/MetaDataEndpointImpl.java       | 33 ++++++++-------
 .../coprocessor/PhoenixAccessController.java    |  8 +---
 3 files changed, 64 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a67869eb/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
new file mode 100644
index 0000000..823605d
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
@@ -0,0 +1,43 @@
+/*
+ * 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.assertFalse;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+
+import org.junit.Test;
+
+public class DropTableIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testRepeatedDropTable() throws Exception {
+      final String tableName = generateUniqueName();
+      final String url = getUrl();
+      try (final Connection conn = DriverManager.getConnection(url);
+          final Statement stmt = conn.createStatement()) {
+        assertFalse(stmt.execute(String.format("CREATE TABLE %s(pk varchar not null primary key)", tableName)));
+        String dropTable = String.format("DROP TABLE IF EXISTS %s", tableName);
+        for (int i = 0; i < 5; i++) {
+          assertFalse(stmt.execute(dropTable));
+        }
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a67869eb/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 11e5e45..05ad959 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
@@ -1293,6 +1293,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return function.getFunctionName() == null;
     }
 
+    private PTable getTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
+            long clientTimeStamp, long asOfTimeStamp, int clientVersion) throws IOException, SQLException {
+        PTable table = loadTable(env, key, cacheKey, clientTimeStamp, asOfTimeStamp, clientVersion);
+        if (table == null || isTableDeleted(table)) { return null; }
+        return table;
+    }
+
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
         throws IOException, SQLException {
@@ -1464,7 +1471,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                    PTable parentTable = getTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
                             clientTimeStamp, clientTimeStamp, clientVersion);
                     if (parentTable == null) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
@@ -1479,13 +1486,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         byte[] parentKey = SchemaUtil.getTableKey(
                                 parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
                                 parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                        parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                 clientTimeStamp, clientTimeStamp, clientVersion);
                         if (parentTable == null) {
                             // it could be a global view
                             parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                                     parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                            parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                     clientTimeStamp, clientTimeStamp, clientVersion);
                         }
                     }
@@ -1969,12 +1976,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] key =
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
-
-            
+            Region region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
             PTableType ptableType=PTableType.fromSerializedValue(tableType);
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
             byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
-            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+            PTable loadedTable = getTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
                     request.getClientVersion());
             if (loadedTable == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
@@ -1986,13 +1997,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     SchemaUtil.getTableName(schemaName, tableName),
                     TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
                     getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
-
-            Region region = env.getRegion();
-            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
-            if (result != null) {
-                done.run(MetaDataMutationResult.toProto(result));
-                return;
-            }
             List<RowLock> locks = Lists.newArrayList();
 
             try {
@@ -2000,6 +2004,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (key != lockKey) {
                     acquireLock(region, key, locks);
                 }
+
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName,
@@ -3657,7 +3662,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
                 //check permission on data table
                 long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                PTable loadedTable = getTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
                         request.getClientVersion());
                 if (loadedTable == null) {
                     builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a67869eb/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 7b9452d..62c158c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -22,7 +22,6 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -67,7 +66,6 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 
-import com.google.common.collect.Lists;
 import com.google.protobuf.RpcCallback;
 
 public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
@@ -102,10 +100,8 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     @Override
     public void preGetTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
             String tableName, TableName physicalTableName) throws IOException {
-        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
-            observer.preGetTableDescriptors(new ObserverContext<MasterCoprocessorEnvironment>(),
-                    Lists.newArrayList(physicalTableName), Collections.<HTableDescriptor> emptyList());
-        }
+        if (!accessCheckEnabled) { return; }
+        requireAccess("GetTable" + tenantId, physicalTableName, Action.READ, Action.EXEC);
     }
 
     @Override


[4/4] phoenix git commit: PHOENIX-4661 Handled deleted PTables in the MetadataCache

Posted by el...@apache.org.
PHOENIX-4661 Handled deleted PTables in the MetadataCache

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Sergey Soldatov <ss...@apache.org>


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

Branch: refs/heads/4.x-HBase-1.2
Commit: c9e46a6fcd9c341185be514ef08a420f5b7f43e4
Parents: 332caa0
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Mar 20 14:54:05 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Mar 20 18:19:06 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DropTableIT.java | 43 ++++++++++++++++++++
 .../coprocessor/MetaDataEndpointImpl.java       | 33 ++++++++-------
 .../coprocessor/PhoenixAccessController.java    |  8 +---
 3 files changed, 64 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9e46a6f/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
new file mode 100644
index 0000000..823605d
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
@@ -0,0 +1,43 @@
+/*
+ * 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.assertFalse;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+
+import org.junit.Test;
+
+public class DropTableIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testRepeatedDropTable() throws Exception {
+      final String tableName = generateUniqueName();
+      final String url = getUrl();
+      try (final Connection conn = DriverManager.getConnection(url);
+          final Statement stmt = conn.createStatement()) {
+        assertFalse(stmt.execute(String.format("CREATE TABLE %s(pk varchar not null primary key)", tableName)));
+        String dropTable = String.format("DROP TABLE IF EXISTS %s", tableName);
+        for (int i = 0; i < 5; i++) {
+          assertFalse(stmt.execute(dropTable));
+        }
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9e46a6f/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 11e5e45..05ad959 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
@@ -1293,6 +1293,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return function.getFunctionName() == null;
     }
 
+    private PTable getTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
+            long clientTimeStamp, long asOfTimeStamp, int clientVersion) throws IOException, SQLException {
+        PTable table = loadTable(env, key, cacheKey, clientTimeStamp, asOfTimeStamp, clientVersion);
+        if (table == null || isTableDeleted(table)) { return null; }
+        return table;
+    }
+
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
         throws IOException, SQLException {
@@ -1464,7 +1471,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                    PTable parentTable = getTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
                             clientTimeStamp, clientTimeStamp, clientVersion);
                     if (parentTable == null) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
@@ -1479,13 +1486,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         byte[] parentKey = SchemaUtil.getTableKey(
                                 parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
                                 parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                        parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                 clientTimeStamp, clientTimeStamp, clientVersion);
                         if (parentTable == null) {
                             // it could be a global view
                             parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                                     parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                            parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                     clientTimeStamp, clientTimeStamp, clientVersion);
                         }
                     }
@@ -1969,12 +1976,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] key =
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
-
-            
+            Region region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
             PTableType ptableType=PTableType.fromSerializedValue(tableType);
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
             byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
-            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+            PTable loadedTable = getTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
                     request.getClientVersion());
             if (loadedTable == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
@@ -1986,13 +1997,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     SchemaUtil.getTableName(schemaName, tableName),
                     TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
                     getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
-
-            Region region = env.getRegion();
-            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
-            if (result != null) {
-                done.run(MetaDataMutationResult.toProto(result));
-                return;
-            }
             List<RowLock> locks = Lists.newArrayList();
 
             try {
@@ -2000,6 +2004,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (key != lockKey) {
                     acquireLock(region, key, locks);
                 }
+
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName,
@@ -3657,7 +3662,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
                 //check permission on data table
                 long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                PTable loadedTable = getTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
                         request.getClientVersion());
                 if (loadedTable == null) {
                     builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9e46a6f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 7b9452d..62c158c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -22,7 +22,6 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -67,7 +66,6 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 
-import com.google.common.collect.Lists;
 import com.google.protobuf.RpcCallback;
 
 public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
@@ -102,10 +100,8 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     @Override
     public void preGetTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
             String tableName, TableName physicalTableName) throws IOException {
-        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
-            observer.preGetTableDescriptors(new ObserverContext<MasterCoprocessorEnvironment>(),
-                    Lists.newArrayList(physicalTableName), Collections.<HTableDescriptor> emptyList());
-        }
+        if (!accessCheckEnabled) { return; }
+        requireAccess("GetTable" + tenantId, physicalTableName, Action.READ, Action.EXEC);
     }
 
     @Override


[2/4] phoenix git commit: PHOENIX-4661 Handled deleted PTables in the MetadataCache

Posted by el...@apache.org.
PHOENIX-4661 Handled deleted PTables in the MetadataCache

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Sergey Soldatov <ss...@apache.org>


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

Branch: refs/heads/master
Commit: e9324cc811e8763f81475af15c46fd739dec26a4
Parents: b6e33f3
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Mar 20 14:54:05 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Mar 20 17:57:16 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DropTableIT.java | 43 ++++++++++++++++++++
 .../coprocessor/MetaDataEndpointImpl.java       | 33 ++++++++-------
 .../coprocessor/PhoenixAccessController.java    |  8 +---
 3 files changed, 64 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e9324cc8/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
new file mode 100644
index 0000000..823605d
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableIT.java
@@ -0,0 +1,43 @@
+/*
+ * 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.assertFalse;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+
+import org.junit.Test;
+
+public class DropTableIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testRepeatedDropTable() throws Exception {
+      final String tableName = generateUniqueName();
+      final String url = getUrl();
+      try (final Connection conn = DriverManager.getConnection(url);
+          final Statement stmt = conn.createStatement()) {
+        assertFalse(stmt.execute(String.format("CREATE TABLE %s(pk varchar not null primary key)", tableName)));
+        String dropTable = String.format("DROP TABLE IF EXISTS %s", tableName);
+        for (int i = 0; i < 5; i++) {
+          assertFalse(stmt.execute(dropTable));
+        }
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e9324cc8/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 11e5e45..05ad959 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
@@ -1293,6 +1293,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return function.getFunctionName() == null;
     }
 
+    private PTable getTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
+            long clientTimeStamp, long asOfTimeStamp, int clientVersion) throws IOException, SQLException {
+        PTable table = loadTable(env, key, cacheKey, clientTimeStamp, asOfTimeStamp, clientVersion);
+        if (table == null || isTableDeleted(table)) { return null; }
+        return table;
+    }
+
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
         throws IOException, SQLException {
@@ -1464,7 +1471,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                    PTable parentTable = getTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
                             clientTimeStamp, clientTimeStamp, clientVersion);
                     if (parentTable == null) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
@@ -1479,13 +1486,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         byte[] parentKey = SchemaUtil.getTableKey(
                                 parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
                                 parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                        parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                 clientTimeStamp, clientTimeStamp, clientVersion);
                         if (parentTable == null) {
                             // it could be a global view
                             parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                                     parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                            parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
                                     clientTimeStamp, clientTimeStamp, clientVersion);
                         }
                     }
@@ -1969,12 +1976,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] key =
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
-
-            
+            Region region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
             PTableType ptableType=PTableType.fromSerializedValue(tableType);
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
             byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
-            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+            PTable loadedTable = getTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
                     request.getClientVersion());
             if (loadedTable == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
@@ -1986,13 +1997,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     SchemaUtil.getTableName(schemaName, tableName),
                     TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
                     getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
-
-            Region region = env.getRegion();
-            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
-            if (result != null) {
-                done.run(MetaDataMutationResult.toProto(result));
-                return;
-            }
             List<RowLock> locks = Lists.newArrayList();
 
             try {
@@ -2000,6 +2004,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (key != lockKey) {
                     acquireLock(region, key, locks);
                 }
+
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName,
@@ -3657,7 +3662,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
                 //check permission on data table
                 long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                PTable loadedTable = getTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
                         request.getClientVersion());
                 if (loadedTable == null) {
                     builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e9324cc8/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 0a07f9e..96467f2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -22,7 +22,6 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -67,7 +66,6 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 
-import com.google.common.collect.Lists;
 import com.google.protobuf.RpcCallback;
 
 public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
@@ -102,10 +100,8 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     @Override
     public void preGetTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
             String tableName, TableName physicalTableName) throws IOException {
-        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
-            observer.preGetTableDescriptors(new ObserverContext<MasterCoprocessorEnvironment>(),
-                    Lists.newArrayList(physicalTableName), Collections.<HTableDescriptor> emptyList());
-        }
+        if (!accessCheckEnabled) { return; }
+        requireAccess("GetTable" + tenantId, physicalTableName, Action.READ, Action.EXEC);
     }
 
     @Override