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 2014/08/15 02:36:45 UTC

git commit: PHOENIX-1171 Dropping the index is not verifying the associated table

Repository: phoenix
Updated Branches:
  refs/heads/3.0 9173fa2e8 -> 3a6a9ddd8


PHOENIX-1171 Dropping the index is not verifying the associated table


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

Branch: refs/heads/3.0
Commit: 3a6a9ddd80d7e97a55e6bd453dab078624e934d5
Parents: 9173fa2
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Aug 14 12:58:12 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Aug 14 16:40:48 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/ViewIT.java | 37 ++++++++++++++++++++
 .../phoenix/end2end/index/IndexMetadataIT.java  | 12 ++++++-
 .../coprocessor/MetaDataEndpointImpl.java       | 14 +++++---
 .../org/apache/phoenix/util/MetaDataUtil.java   |  4 ++-
 .../phoenix/compile/ViewCompilerTest.java       | 12 +++----
 5 files changed, 66 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3a6a9ddd/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index b75bb30..394fa04 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -29,6 +29,7 @@ import java.sql.SQLException;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -233,4 +234,40 @@ public class ViewIT extends BaseViewIT {
         }
         assertEquals(4, count);
     }
+    
+    @Test
+    public void testViewAndTableInDifferentSchemas() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE s1.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM s1.t WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW v2 (v2 VARCHAR) AS SELECT * FROM s1.t WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        ddl = "DROP VIEW v1";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (TableNotFoundException ignore) {
+        }
+        ddl = "DROP VIEW s2.v1";
+        conn.createStatement().execute(ddl);
+        ddl = "DROP VIEW s2.v2";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (TableNotFoundException ignore) {
+        }
+        ddl = "DROP TABLE s1.t";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+        }
+        ddl = "DROP VIEW v2";
+        conn.createStatement().execute(ddl);
+        ddl = "DROP TABLE s1.t";
+        conn.createStatement().execute(ddl);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3a6a9ddd/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index 35232b5..2547844 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -45,6 +45,7 @@ import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
@@ -115,7 +116,7 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testIndexCreation() throws Exception {
+    public void testIndexCreateDrop() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
@@ -259,6 +260,15 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
             assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 8, "B:INT_COL2", null);
             assertFalse(rs.next());
             
+            // Create another table in the same schema
+            String diffTableNameInSameSchema = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + "2";
+            conn.createStatement().execute("CREATE TABLE " + diffTableNameInSameSchema + "(k INTEGER PRIMARY KEY)");
+            try {
+                conn.createStatement().execute("DROP INDEX IDX1 ON " + diffTableNameInSameSchema);
+                fail("Should have realized index IDX1 is not on the table");
+            } catch (TableNotFoundException ignore) {
+                
+            }
             ddl = "DROP TABLE " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
             stmt = conn.prepareStatement(ddl);
             stmt.execute();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3a6a9ddd/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 aae98b2..07d4cc8 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
@@ -665,7 +665,7 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
                     acquireLock(region, key, lids);
                 }
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
-                result = doDropTable(key, tenantIdBytes, schemaName, tableName, PTableType.fromSerializedValue(tableType), tableMetadata, invalidateList, lids, tableNamesToDelete);
+                result = doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName, PTableType.fromSerializedValue(tableType), tableMetadata, invalidateList, lids, tableNamesToDelete);
                 if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                     return result;
                 }
@@ -690,8 +690,8 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
         }
     }
 
-    private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName, PTableType tableType, 
-            List<Mutation> rowsToDelete, List<ImmutableBytesPtr> invalidateList, List<Integer> lids, List<byte[]> tableNamesToDelete) throws IOException, SQLException {
+    private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName, byte[] parentTableName, 
+            PTableType tableType, List<Mutation> rowsToDelete, List<ImmutableBytesPtr> invalidateList, List<Integer> lids, List<byte[]> tableNamesToDelete) throws IOException, SQLException {
         long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete);
 
         RegionCoprocessorEnvironment env = getEnvironment();
@@ -720,6 +720,10 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
             }
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
         }
+        // Make sure we're not deleting the "wrong" child
+        if (!Arrays.equals(parentTableName, table.getParentTableName() == null ? null : table.getParentTableName().getBytes())) {
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
         // Since we don't allow back in time DDL, we know if we have a table it's the one
         // we want to delete. FIXME: we shouldn't need a scan here, but should be able to
         // use the table to generate the Delete markers.
@@ -769,7 +773,7 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
             Delete delete = new Delete(indexKey, clientTimeStamp, null);
             rowsToDelete.add(delete);
             acquireLock(region, indexKey, lids);
-            MetaDataMutationResult result = doDropTable(indexKey, tenantId, schemaName, indexName, PTableType.INDEX, rowsToDelete, invalidateList, lids, tableNamesToDelete);
+            MetaDataMutationResult result = doDropTable(indexKey, tenantId, schemaName, indexName, tableName, PTableType.INDEX, rowsToDelete, invalidateList, lids, tableNamesToDelete);
             if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                 return result;
             }
@@ -971,7 +975,7 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
                                             byte[] linkKey = MetaDataUtil.getParentLinkKey(tenantId, schemaName, tableName, index.getTableName().getBytes());
                                             // Drop the link between the data table and the index table
                                             additionalTableMetaData.add(new Delete(linkKey, clientTimeStamp, null));
-                                            doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes(), index.getType(), additionalTableMetaData, invalidateList, lids, tableNamesToDelete);
+                                            doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes(), tableName, index.getType(), additionalTableMetaData, invalidateList, lids, tableNamesToDelete);
                                             // TODO: return in result?
                                         } else {
                                             invalidateList.add(new ImmutableBytesPtr(indexKey));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3a6a9ddd/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 4468414..57ef77d 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
@@ -130,10 +130,12 @@ public class MetaDataUtil {
         }
         byte[][] rowKeyMetaData = new byte[3][];
         getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+        byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
         byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         Mutation m = getParentTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
-        if (Bytes.compareTo(tableName, rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
+        if (   Bytes.compareTo(schemaName, rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]) == 0
+            && Bytes.compareTo(tableName, rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
             return null;
         }
         return rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3a6a9ddd/phoenix-core/src/test/java/org/apache/phoenix/compile/ViewCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/ViewCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/ViewCompilerTest.java
index 651d58d..7a0bac6 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/ViewCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/ViewCompilerTest.java
@@ -80,25 +80,25 @@ public class ViewCompilerTest extends BaseConnectionlessQueryTest {
     public void testViewInvalidation() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         PhoenixConnection conn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
-        String ct = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 VARCHAR, v VARCHAR, CONSTRAINT pk PRIMARY KEY (k1,k2))";
+        String ct = "CREATE TABLE s1.t (k1 INTEGER NOT NULL, k2 VARCHAR, v VARCHAR, CONSTRAINT pk PRIMARY KEY (k1,k2))";
         conn.createStatement().execute(ct);
-        conn.createStatement().execute("CREATE VIEW v3 AS SELECT * FROM t WHERE v = 'bar'");
+        conn.createStatement().execute("CREATE VIEW s2.v3 AS SELECT * FROM s1.t WHERE v = 'bar'");
         
         // TODO: should it be an error to remove columns from a VIEW that we're defined there?
         // TOOD: should we require an ALTER VIEW instead of ALTER TABLE?
-        conn.createStatement().execute("ALTER VIEW v3 DROP COLUMN v");
+        conn.createStatement().execute("ALTER VIEW s2.v3 DROP COLUMN v");
         try {
-            conn.createStatement().executeQuery("SELECT * FROM v3");
+            conn.createStatement().executeQuery("SELECT * FROM s2.v3");
             fail();
         } catch (ColumnNotFoundException e) {
             
         }
         
         // No error, as v still exists in t
-        conn.createStatement().execute("CREATE VIEW v4 AS SELECT * FROM t WHERE v = 'bas'");
+        conn.createStatement().execute("CREATE VIEW s2.v4 AS SELECT * FROM s1.t WHERE v = 'bas'");
 
         // No error, even though view is invalid
-        conn.createStatement().execute("DROP VIEW v3");
+        conn.createStatement().execute("DROP VIEW s2.v3");
     }