Repository: phoenix Updated Branches: refs/heads/4.0 2051459cd -> a6ea5560e
PHOENIX-1098 Support CASCADE option on DROP TABLE that drops all VIEWs (Jan Fernando) Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/a6ea5560 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/a6ea5560 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/a6ea5560 Branch: refs/heads/4.0 Commit: a6ea5560e52d2a0f6a4c9ca4fca585a9225a0f75 Parents: 2051459 Author: James Taylor <jtay...@salesforce.com> Authored: Thu Sep 4 00:13:27 2014 -0700 Committer: James Taylor <jtay...@salesforce.com> Committed: Thu Sep 4 00:13:27 2014 -0700 ---------------------------------------------------------------------- .../end2end/SkipScanAfterManualSplitIT.java | 27 +++ .../end2end/TenantSpecificTablesDDLIT.java | 108 ++++++++++- .../java/org/apache/phoenix/end2end/ViewIT.java | 90 +++++++++- phoenix-core/src/main/antlr3/PhoenixSQL.g | 5 +- .../coprocessor/MetaDataEndpointImpl.java | 129 ++++++++++++-- .../coprocessor/generated/MetaDataProtos.java | 178 +++++++++++++------ .../coprocessor/generated/PTableProtos.java | 21 --- .../generated/ServerCacheFactoryProtos.java | 21 --- .../generated/ServerCachingProtos.java | 21 --- .../apache/phoenix/jdbc/PhoenixStatement.java | 8 +- .../phoenix/parse/DropTableStatement.java | 9 +- .../apache/phoenix/parse/ParseNodeFactory.java | 4 +- .../phoenix/query/ConnectionQueryServices.java | 2 +- .../query/ConnectionQueryServicesImpl.java | 3 +- .../query/ConnectionlessQueryServicesImpl.java | 2 +- .../query/DelegateConnectionQueryServices.java | 4 +- .../apache/phoenix/schema/MetaDataClient.java | 8 +- phoenix-protocol/src/main/MetaDataService.proto | 1 + 18 files changed, 488 insertions(+), 153 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java index 6e8ffb7..f7a2448 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java @@ -346,4 +346,31 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT { assertFalse(rs.next()); } + + @Test + public void testMinMaxRangeIntersection() throws Exception { + Connection conn = DriverManager.getConnection(getUrl()); + + PreparedStatement stmt = conn.prepareStatement("create table splits_test " + + "(pk1 UNSIGNED_TINYINT NOT NULL, pk2 UNSIGNED_TINYINT NOT NULL, kv VARCHAR " + + "CONSTRAINT pk PRIMARY KEY (pk1, pk2)) SALT_BUCKETS=4 SPLIT ON (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"); + // Split each salt bucket into multiple regions + stmt.setBytes(1, new byte[] {0, 1, 1}); + stmt.setBytes(2, new byte[] {0, 2, 1}); + stmt.setBytes(3, new byte[] {0, 3, 1}); + stmt.setBytes(4, new byte[] {1, 1, 1}); + stmt.setBytes(5, new byte[] {1, 2, 1}); + stmt.setBytes(6, new byte[] {1, 3, 1}); + stmt.setBytes(7, new byte[] {2, 1, 1}); + stmt.setBytes(8, new byte[] {2, 2, 1}); + stmt.setBytes(9, new byte[] {2, 3, 1}); + stmt.setBytes(10, new byte[] {3, 1, 1}); + stmt.setBytes(11, new byte[] {3, 2, 1}); + stmt.setBytes(12, new byte[] {3, 3, 1}); + stmt.execute(); + + // Use a query with a RVC in a non equality expression + ResultSet rs = conn.createStatement().executeQuery("select count(kv) from splits_test where pk1 < 3 and (pk1,PK2) >= (3, 1)"); + assertTrue(rs.next()); + } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java index 79aa6c1..591efe1 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java @@ -331,7 +331,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT { } @Test - public void testDropParentTableWithExistingTenantTable() throws Exception { + public void testDisallowDropParentTableWithExistingTenantTable() throws Exception { Properties props = new Properties(); props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp())); Connection conn = DriverManager.getConnection(getUrl(), props); @@ -348,6 +348,112 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT { } @Test + public void testAllowDropParentTableWithCascadeAndSingleTenantTable() throws Exception { + long ts = nextTimestamp(); + Properties props = new Properties(); + props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts)); + Connection conn = DriverManager.getConnection(getUrl(), props); + Connection connTenant = null; + + try { + // Drop Parent Table + conn.createStatement().executeUpdate("DROP TABLE " + PARENT_TABLE_NAME + " CASCADE"); + conn.close(); + + props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10)); + connTenant = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props); + + validateTenantViewIsDropped(conn); + } finally { + if (conn != null) { + conn.close(); + } + if (connTenant != null) { + connTenant.close(); + } + } + } + + + @Test + public void testAllDropParentTableWithCascadeWithMultipleTenantTablesAndIndexes() throws Exception { + // Create a second tenant table + createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL2, TENANT_TABLE_DDL, null, nextTimestamp()); + //TODO Create some tenant specific table indexes + + long ts = nextTimestamp(); + Properties props = new Properties(); + props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts)); + Connection conn = null; + Connection connTenant1 = null; + Connection connTenant2 = null; + + try { + conn = DriverManager.getConnection(getUrl(), props); + DatabaseMetaData meta = conn.getMetaData(); + ResultSet rs = meta.getSuperTables(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME) + "%"); + assertTrue(rs.next()); + assertEquals(TENANT_ID2, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); + assertEquals(TENANT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); + assertEquals(PARENT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); + assertTrue(rs.next()); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); + assertEquals(TENANT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); + assertEquals(PARENT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); + assertTrue(rs.next()); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); + assertEquals(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); + assertEquals(PARENT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); + assertFalse(rs.next()); + rs.close(); + conn.close(); + + // Drop Parent Table + conn.createStatement().executeUpdate("DROP TABLE " + PARENT_TABLE_NAME + " CASCADE"); + + // Validate Tenant Views are dropped + props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10)); + connTenant1 = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props); + validateTenantViewIsDropped(connTenant1); + connTenant2 = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2, props); + validateTenantViewIsDropped(connTenant2); + + // Validate Tenant Metadata is gone for the Tenant Table TENANT_TABLE_NAME + conn = DriverManager.getConnection(getUrl(), props); + meta = conn.getMetaData(); + rs = meta.getSuperTables(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME) + "%"); + assertTrue(rs.next()); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); + assertEquals(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); + assertEquals(PARENT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); + assertFalse(rs.next()); + rs.close(); + + } finally { + if (conn != null) { + conn.close(); + } + if (connTenant1 != null) { + connTenant1.close(); + } + if (connTenant2 != null) { + connTenant2.close(); + } + } + } + + private void validateTenantViewIsDropped(Connection connTenant) throws SQLException { + // Try and drop tenant view, should throw TableNotFoundException + try { + String ddl = "DROP VIEW " + TENANT_TABLE_NAME; + connTenant.createStatement().execute(ddl); + fail("Tenant specific view " + TENANT_TABLE_NAME + " should have been dropped when parent was dropped"); + } catch (TableNotFoundException e) { + //Expected + } + } + + @Test public void testTableMetadataScan() throws Exception { // create a tenant table with same name for a different tenant to make sure we are not picking it up in metadata scans for TENANT_ID String tenantId2 = "tenant2"; http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 d79535a..20f606d 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 @@ -259,15 +259,99 @@ public class ViewIT extends BaseViewIT { } catch (TableNotFoundException ignore) { } ddl = "DROP TABLE s1.t"; + validateCannotDropTableWithChildViewsWithoutCascade(conn, "s1.t"); + ddl = "DROP VIEW v2"; + conn.createStatement().execute(ddl); + ddl = "DROP TABLE s1.t"; + conn.createStatement().execute(ddl); + } + + + @Test + public void testDisallowDropOfColumnOnParentTable() throws Exception { + Connection conn = DriverManager.getConnection(getUrl()); + String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))"; + conn.createStatement().execute(ddl); + ddl = "CREATE VIEW v1(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM tp WHERE v1 = 1.0"; + conn.createStatement().execute(ddl); + try { - conn.createStatement().execute(ddl); + conn.createStatement().execute("ALTER TABLE tp DROP COLUMN v1"); fail(); } catch (SQLException e) { assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); } - ddl = "DROP VIEW v2"; + } + + @Test + public void testViewAndTableAndDropCascade() throws Exception { + // Setup + Connection conn = DriverManager.getConnection(getUrl()); + String ddl = "CREATE TABLE s2.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"; conn.createStatement().execute(ddl); - ddl = "DROP TABLE s1.t"; + ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM s2.t WHERE k > 5"; + conn.createStatement().execute(ddl); + ddl = "CREATE VIEW s2.v2 (v2 VARCHAR) AS SELECT * FROM s2.t WHERE k > 10"; + conn.createStatement().execute(ddl); + + validateCannotDropTableWithChildViewsWithoutCascade(conn, "s2.t"); + + // Execute DROP...CASCADE + conn.createStatement().execute("DROP TABLE s2.t CASCADE"); + + validateViewDoesNotExist(conn, "s2.v1"); + validateViewDoesNotExist(conn, "s2.v2"); + } + + @Test + public void testViewAndTableAndDropCascadeWithIndexes() throws Exception { + + // Setup - Tables and Views with Indexes + Connection conn = DriverManager.getConnection(getUrl()); + + String ddl = "CREATE TABLE s3.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) IMMUTABLE_ROWS=true"; + conn.createStatement().execute(ddl); + ddl = "CREATE INDEX IDX1 ON s3.t (v1)"; + conn.createStatement().execute(ddl); + ddl = "CREATE VIEW s3.v1 (v2 VARCHAR) AS SELECT * FROM s3.t WHERE k > 5"; + conn.createStatement().execute(ddl); + ddl = "CREATE INDEX IDX2 ON s3.v1 (v2)"; + conn.createStatement().execute(ddl); + ddl = "CREATE VIEW s3.v2 (v2 VARCHAR) AS SELECT * FROM s3.t WHERE k > 10"; + conn.createStatement().execute(ddl); + ddl = "CREATE INDEX IDX3 ON s3.v2 (v2)"; conn.createStatement().execute(ddl); + + validateCannotDropTableWithChildViewsWithoutCascade(conn, "s3.t"); + + // Execute DROP...CASCADE + conn.createStatement().execute("DROP TABLE s3.t CASCADE"); + + // Validate Views were deleted - Try and delete child views, should throw TableNotFoundException + validateViewDoesNotExist(conn, "s3.v1"); + validateViewDoesNotExist(conn, "s3.v2"); } + + + private void validateCannotDropTableWithChildViewsWithoutCascade(Connection conn, String tableName) throws SQLException { + String ddl; + try { + ddl = "DROP TABLE " + tableName; + conn.createStatement().execute(ddl); + fail("Should not be able to drop table " + tableName + " with child views without explictly specifying CASCADE"); + } catch (SQLException e) { + assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); + } + } + + + private void validateViewDoesNotExist(Connection conn, String viewName) throws SQLException { + try { + String ddl1 = "DROP VIEW " + viewName; + conn.createStatement().execute(ddl1); + fail("View s3.v1 should have been deleted when parent was dropped"); + } catch (TableNotFoundException e) { + //Expected + } + } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/antlr3/PhoenixSQL.g ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g index 80f1238..556cd21 100644 --- a/phoenix-core/src/main/antlr3/PhoenixSQL.g +++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g @@ -104,6 +104,7 @@ tokens MINVALUE='minvalue'; MAXVALUE='maxvalue'; CYCLE='cycle'; + CASCADE='cascade'; } @@ -470,8 +471,8 @@ column_names returns [List<ColumnName> ret] // Parse a drop table statement. drop_table_node returns [DropTableStatement ret] - : DROP (v=VIEW | TABLE) (IF ex=EXISTS)? t=from_table_name - {ret = factory.dropTable(t, v==null ? PTableType.TABLE : PTableType.VIEW, ex!=null); } + : DROP (v=VIEW | TABLE) (IF ex=EXISTS)? t=from_table_name (c=CASCADE)? + {ret = factory.dropTable(t, v==null ? PTableType.TABLE : PTableType.VIEW, ex!=null, c!=null); } ; // Parse a drop index statement http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 020a3c9..2efedb3 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 @@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion.RowLock; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -129,6 +130,7 @@ import org.apache.phoenix.schema.PTableImpl; import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.schema.TableNotFoundException; +import org.apache.phoenix.schema.tuple.ResultTuple; import org.apache.phoenix.trace.util.Tracing; import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.IndexUtil; @@ -861,11 +863,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso protected static final byte[] PHYSICAL_TABLE_BYTES = new byte[] {PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()}; /** * @param tableName parent table's name - * @return true if there exist a table that use this table as their base table. + * Looks for whether child views exist for the table specified by table. * TODO: should we pass a timestamp here? */ @SuppressWarnings("deprecation") - private boolean hasViews(HRegion region, byte[] tenantId, PTable table) throws IOException { + private TableViewFinderResult findChildViews(HRegion region, byte[] tenantId, PTable table) throws IOException { byte[] schemaName = table.getSchemaName().getBytes(); byte[] tableName = table.getTableName().getBytes(); boolean isMultiTenant = table.isMultiTenant(); @@ -895,18 +897,31 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso HTablePool pool = new HTablePool (env.getConfiguration(),1); try { HTableInterface hTable = pool.getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME); + ResultScanner scanner = hTable.getScanner(scan); + boolean allViewsInCurrentRegion = true; + int numOfChildViews = 0; + List<Result> results = Lists.newArrayList(); try { - ResultScanner scanner = hTable.getScanner(scan); - try { - Result result = scanner.next(); - return result != null; - } - finally { - scanner.close(); + for (Result result = scanner.next(); (result != null); result = scanner.next()) { + numOfChildViews++; + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + ResultTuple resultTuple = new ResultTuple(result); + resultTuple.getKey(ptr); + byte[] key = ptr.copyBytes(); + if (checkTableKeyInRegion(key, region) != null) { + allViewsInCurrentRegion = false; + } + results.add(result); } } finally { + scanner.close(); hTable.close(); } + TableViewFinderResult tableViewFinderResult = new TableViewFinderResult(results); + if (numOfChildViews > 0 && !allViewsInCurrentRegion) { + tableViewFinderResult.setAllViewsNotInSingleRegion(); + } + return tableViewFinderResult; } finally { pool.close(); } @@ -916,6 +931,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso public void dropTable(RpcController controller, DropTableRequest request, RpcCallback<MetaDataResponse> done) { MetaDataResponse.Builder builder = MetaDataResponse.newBuilder(); + boolean isCascade = request.getCascade(); byte[][] rowKeyMetaData = new byte[3][]; String tableType = request.getTableType(); byte[] schemaName = null; @@ -949,6 +965,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso return; } List<RowLock> locks = Lists.newArrayList(); + try { acquireLock(region, lockKey, locks); if (key != lockKey) { @@ -958,7 +975,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso result = doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName, PTableType.fromSerializedValue(tableType), tableMetadata, - invalidateList, locks, tableNamesToDelete); + invalidateList, locks, tableNamesToDelete, isCascade); if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) { done.run(MetaDataMutationResult.toProto(result)); return; @@ -989,7 +1006,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName, byte[] parentTableName, PTableType tableType, List<Mutation> rowsToDelete, List<ImmutableBytesPtr> invalidateList, List<RowLock> locks, - List<byte[]> tableNamesToDelete) throws IOException, SQLException { + List<byte[]> tableNamesToDelete, boolean isCascade) throws IOException, SQLException { + + long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete); HRegion region = env.getRegion(); @@ -997,7 +1016,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache(); PTable table = metaDataCache.getIfPresent(cacheKey); - + // We always cache the latest version - fault in if not in cache if (table != null || (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP)) != null) { @@ -1032,9 +1051,41 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso if (results.isEmpty()) { // Should not be possible return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null); } - if (hasViews(region, tenantId, table)) { - return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); - } + + // Handle any child views that exist + TableViewFinderResult tableViewFinderResult = findChildViews(region, tenantId, table); + if (tableViewFinderResult.hasViews()) { + if (isCascade) { + if (tableViewFinderResult.allViewsInMultipleRegions()) { + // We don't yet support deleting a table with views where SYSTEM.CATALOG has split and the + // view metadata spans multiple regions + return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); + } else if (tableViewFinderResult.allViewsInSingleRegion()) { + // Recursively delete views - safe as all the views as all in the same region + for (Result viewResult : tableViewFinderResult.getResults()) { + byte[][] rowKeyMetaData = new byte[3][]; + getVarChars(viewResult.getRow(), 3, rowKeyMetaData); + byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX]; + byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]; + byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]; + byte[] viewKey = SchemaUtil.getTableKey(viewTenantId, viewSchemaName, viewName); + Delete delete = new Delete(viewKey, clientTimeStamp); + rowsToDelete.add(delete); + acquireLock(region, viewKey, locks); + MetaDataMutationResult result = + doDropTable(viewKey, viewTenantId, viewSchemaName, viewName, null, PTableType.VIEW, + rowsToDelete, invalidateList, locks, tableNamesToDelete, false); + if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) { + return result; + } + } + } + } else { + // DROP without CASCADE on tables with child views is not permitted + return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); + } + } + if (tableType != PTableType.VIEW) { // Add to list of HTables to delete, unless it's a view tableNamesToDelete.add(table.getName().getBytes()); } @@ -1073,7 +1124,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso acquireLock(region, indexKey, locks); MetaDataMutationResult result = doDropTable(indexKey, tenantId, schemaName, indexName, tableName, PTableType.INDEX, - rowsToDelete, invalidateList, locks, tableNamesToDelete); + rowsToDelete, invalidateList, locks, tableNamesToDelete, false); if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) { return result; } @@ -1174,7 +1225,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso if (type != expectedType) { return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null); } - if (hasViews(region, tenantId, table)) { + if (findChildViews(region, tenantId, table).hasViews()) { // Disallow any column mutations for parents of tenant tables return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); } @@ -1345,7 +1396,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso // index table additionalTableMetaData.add(new Delete(linkKey, clientTimeStamp)); doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes(), tableName, - index.getType(), additionalTableMetaData, invalidateList, locks, tableNamesToDelete); + index.getType(), additionalTableMetaData, invalidateList, locks, tableNamesToDelete, false); // TODO: return in result? } else { invalidateList.add(new ImmutableBytesPtr(indexKey)); @@ -1579,4 +1630,46 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso return new MetaDataMutationResult(MutationCode.TABLE_NOT_IN_REGION, EnvironmentEdgeManager.currentTimeMillis(), null); } + + /** + * Certain operations, such as DROP TABLE are not allowed if there a table has child views. + * This class wraps the Results of a scanning the Phoenix Metadata for child views for a specific table + * and stores an additional flag for whether whether SYSTEM.CATALOG has split across multiple regions. + */ + private static class TableViewFinderResult { + + private List<Result> results = Lists.newArrayList(); + private boolean allViewsNotInSingleRegion = false; + + private TableViewFinderResult(List<Result> results) { + this.results = results; + } + + public boolean hasViews() { + return results.size() > 0; + } + + private void setAllViewsNotInSingleRegion() { + allViewsNotInSingleRegion = true; + } + + private List<Result> getResults() { + return results; + } + + /** + * Returns true is the table has views and they are all in the same HBase region. + */ + private boolean allViewsInSingleRegion() { + return results.size() > 0 && !allViewsNotInSingleRegion; + } + + /** + * Returns true is the table has views and they are all NOT in the same HBase region. + */ + private boolean allViewsInMultipleRegions() { + return results.size() > 0 && allViewsNotInSingleRegion; + } + } + } http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java index 8c32f14..7877fc4 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java @@ -2,27 +2,6 @@ // source: MetaDataService.proto package org.apache.phoenix.coprocessor.generated; -/* - * - * 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. - * -*/ - public final class MetaDataProtos { private MetaDataProtos() {} @@ -2650,6 +2629,16 @@ public final class MetaDataProtos { */ com.google.protobuf.ByteString getTableTypeBytes(); + + // required bool cascade = 3; + /** + * <code>required bool cascade = 3;</code> + */ + boolean hasCascade(); + /** + * <code>required bool cascade = 3;</code> + */ + boolean getCascade(); } /** * Protobuf type {@code DropTableRequest} @@ -2715,6 +2704,11 @@ public final class MetaDataProtos { tableType_ = input.readBytes(); break; } + case 24: { + bitField0_ |= 0x00000002; + cascade_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -2824,9 +2818,26 @@ public final class MetaDataProtos { } } + // required bool cascade = 3; + public static final int CASCADE_FIELD_NUMBER = 3; + private boolean cascade_; + /** + * <code>required bool cascade = 3;</code> + */ + public boolean hasCascade() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * <code>required bool cascade = 3;</code> + */ + public boolean getCascade() { + return cascade_; + } + private void initFields() { tableMetadataMutations_ = java.util.Collections.emptyList(); tableType_ = ""; + cascade_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2837,6 +2848,10 @@ public final class MetaDataProtos { memoizedIsInitialized = 0; return false; } + if (!hasCascade()) { + memoizedIsInitialized = 0; + return false; + } memoizedIsInitialized = 1; return true; } @@ -2850,6 +2865,9 @@ public final class MetaDataProtos { if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeBytes(2, getTableTypeBytes()); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(3, cascade_); + } getUnknownFields().writeTo(output); } @@ -2872,6 +2890,10 @@ public final class MetaDataProtos { size += com.google.protobuf.CodedOutputStream .computeBytesSize(2, getTableTypeBytes()); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(3, cascade_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2902,6 +2924,11 @@ public final class MetaDataProtos { result = result && getTableType() .equals(other.getTableType()); } + result = result && (hasCascade() == other.hasCascade()); + if (hasCascade()) { + result = result && (getCascade() + == other.getCascade()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2923,6 +2950,10 @@ public final class MetaDataProtos { hash = (37 * hash) + TABLETYPE_FIELD_NUMBER; hash = (53 * hash) + getTableType().hashCode(); } + if (hasCascade()) { + hash = (37 * hash) + CASCADE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getCascade()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -3036,6 +3067,8 @@ public final class MetaDataProtos { bitField0_ = (bitField0_ & ~0x00000001); tableType_ = ""; bitField0_ = (bitField0_ & ~0x00000002); + cascade_ = false; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -3073,6 +3106,10 @@ public final class MetaDataProtos { to_bitField0_ |= 0x00000001; } result.tableType_ = tableType_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000002; + } + result.cascade_ = cascade_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -3104,6 +3141,9 @@ public final class MetaDataProtos { tableType_ = other.tableType_; onChanged(); } + if (other.hasCascade()) { + setCascade(other.getCascade()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -3113,6 +3153,10 @@ public final class MetaDataProtos { return false; } + if (!hasCascade()) { + + return false; + } return true; } @@ -3281,6 +3325,39 @@ public final class MetaDataProtos { return this; } + // required bool cascade = 3; + private boolean cascade_ ; + /** + * <code>required bool cascade = 3;</code> + */ + public boolean hasCascade() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * <code>required bool cascade = 3;</code> + */ + public boolean getCascade() { + return cascade_; + } + /** + * <code>required bool cascade = 3;</code> + */ + public Builder setCascade(boolean value) { + bitField0_ |= 0x00000004; + cascade_ = value; + onChanged(); + return this; + } + /** + * <code>required bool cascade = 3;</code> + */ + public Builder clearCascade() { + bitField0_ = (bitField0_ & ~0x00000004); + cascade_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:DropTableRequest) } @@ -7041,35 +7118,36 @@ public final class MetaDataProtos { "\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022" + "\026\n\016tableTimestamp\030\004 \002(\003\022\027\n\017clientTimesta" + "mp\030\005 \002(\003\"4\n\022CreateTableRequest\022\036\n\026tableM" + - "etadataMutations\030\001 \003(\014\"E\n\020DropTableReque", + "etadataMutations\030\001 \003(\014\"V\n\020DropTableReque", "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\tta" + - "bleType\030\002 \002(\t\"2\n\020AddColumnRequest\022\036\n\026tab" + - "leMetadataMutations\030\001 \003(\014\"3\n\021DropColumnR" + - "equest\022\036\n\026tableMetadataMutations\030\001 \003(\014\"9" + - "\n\027UpdateIndexStateRequest\022\036\n\026tableMetada" + - "taMutations\030\001 \003(\014\"\023\n\021ClearCacheRequest\"\024" + - "\n\022ClearCacheResponse\"\023\n\021GetVersionReques" + - "t\"%\n\022GetVersionResponse\022\017\n\007version\030\001 \002(\003" + - "*\212\002\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXIST" + - "S\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_F", - "OUND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CON" + - "CURRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN" + - "_REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNAL" + - "LOWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020" + - "\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t2\303\003\n\017MetaDa" + - "taService\022/\n\010getTable\022\020.GetTableRequest\032" + - "\021.MetaDataResponse\0225\n\013createTable\022\023.Crea" + - "teTableRequest\032\021.MetaDataResponse\0221\n\tdro" + - "pTable\022\021.DropTableRequest\032\021.MetaDataResp" + - "onse\0221\n\taddColumn\022\021.AddColumnRequest\032\021.M", - "etaDataResponse\0223\n\ndropColumn\022\022.DropColu" + - "mnRequest\032\021.MetaDataResponse\022?\n\020updateIn" + - "dexState\022\030.UpdateIndexStateRequest\032\021.Met" + - "aDataResponse\0225\n\nclearCache\022\022.ClearCache" + - "Request\032\023.ClearCacheResponse\0225\n\ngetVersi" + - "on\022\022.GetVersionRequest\032\023.GetVersionRespo" + - "nseBB\n(org.apache.phoenix.coprocessor.ge" + - "neratedB\016MetaDataProtosH\001\210\001\001\240\001\001" + "bleType\030\002 \002(\t\022\017\n\007cascade\030\003 \002(\010\"2\n\020AddCol" + + "umnRequest\022\036\n\026tableMetadataMutations\030\001 \003" + + "(\014\"3\n\021DropColumnRequest\022\036\n\026tableMetadata" + + "Mutations\030\001 \003(\014\"9\n\027UpdateIndexStateReque" + + "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\"\023\n\021Cl" + + "earCacheRequest\"\024\n\022ClearCacheResponse\"\023\n" + + "\021GetVersionRequest\"%\n\022GetVersionResponse" + + "\022\017\n\007version\030\001 \002(\003*\212\002\n\014MutationCode\022\030\n\024TA" + + "BLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020", + "\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREAD" + + "Y_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020" + + "\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABL" + + "E_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_MUTATION\020\007\022" + + "\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_F" + + "OUND\020\t2\303\003\n\017MetaDataService\022/\n\010getTable\022\020" + + ".GetTableRequest\032\021.MetaDataResponse\0225\n\013c" + + "reateTable\022\023.CreateTableRequest\032\021.MetaDa" + + "taResponse\0221\n\tdropTable\022\021.DropTableReque" + + "st\032\021.MetaDataResponse\0221\n\taddColumn\022\021.Add", + "ColumnRequest\032\021.MetaDataResponse\0223\n\ndrop" + + "Column\022\022.DropColumnRequest\032\021.MetaDataRes" + + "ponse\022?\n\020updateIndexState\022\030.UpdateIndexS" + + "tateRequest\032\021.MetaDataResponse\0225\n\nclearC" + + "ache\022\022.ClearCacheRequest\032\023.ClearCacheRes" + + "ponse\0225\n\ngetVersion\022\022.GetVersionRequest\032" + + "\023.GetVersionResponseBB\n(org.apache.phoen" + + "ix.coprocessor.generatedB\016MetaDataProtos" + + "H\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -7099,7 +7177,7 @@ public final class MetaDataProtos { internal_static_DropTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DropTableRequest_descriptor, - new java.lang.String[] { "TableMetadataMutations", "TableType", }); + new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", }); internal_static_AddColumnRequest_descriptor = getDescriptor().getMessageTypes().get(4); internal_static_AddColumnRequest_fieldAccessorTable = new http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java index 9cb47b9..ef0ece2 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java @@ -2,27 +2,6 @@ // source: PTable.proto package org.apache.phoenix.coprocessor.generated; -/* - * - * 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. - * -*/ - public final class PTableProtos { private PTableProtos() {} http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCacheFactoryProtos.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCacheFactoryProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCacheFactoryProtos.java index ce17c86..09b39d0 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCacheFactoryProtos.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCacheFactoryProtos.java @@ -2,27 +2,6 @@ // source: ServerCacheFactory.proto package org.apache.phoenix.coprocessor.generated; -/* - * - * 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. - * -*/ - public final class ServerCacheFactoryProtos { private ServerCacheFactoryProtos() {} http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java index 97248f4..69db21b 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java @@ -2,27 +2,6 @@ // source: ServerCachingService.proto package org.apache.phoenix.coprocessor.generated; -/* - * - * 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. - * -*/ - public final class ServerCachingProtos { private ServerCachingProtos() {} http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java index 870adc4..00d271d 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java @@ -530,8 +530,8 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho private static class ExecutableDropTableStatement extends DropTableStatement implements CompilableStatement { - ExecutableDropTableStatement(TableName tableName, PTableType tableType, boolean ifExists) { - super(tableName, tableType, ifExists); + ExecutableDropTableStatement(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) { + super(tableName, tableType, ifExists, cascade); } @SuppressWarnings("unchecked") @@ -788,8 +788,8 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho } @Override - public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists) { - return new ExecutableDropTableStatement(tableName, tableType, ifExists); + public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) { + return new ExecutableDropTableStatement(tableName, tableType, ifExists, cascade); } @Override http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java index 2945d36..997b695 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java @@ -24,11 +24,14 @@ public class DropTableStatement extends MutableStatement { private final TableName tableName; private final boolean ifExists; private final PTableType tableType; + private final boolean cascade; + - protected DropTableStatement(TableName tableName, PTableType tableType, boolean ifExists) { + protected DropTableStatement(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) { this.tableName = tableName; this.tableType = tableType; this.ifExists = ifExists; + this.cascade = cascade; } @Override @@ -48,6 +51,10 @@ public class DropTableStatement extends MutableStatement { return ifExists; } + public boolean cascade() { + return cascade; + } + @Override public Operation getOperation() { return Operation.DELETE; http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java index 6872f8a..e16849f 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java @@ -300,8 +300,8 @@ public class ParseNodeFactory { return new DropColumnStatement(table, tableType, columnNodes, ifExists); } - public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists) { - return new DropTableStatement(tableName, tableType, ifExists); + public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) { + return new DropTableStatement(tableName, tableType, ifExists, cascade); } public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) { http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 2470e5d..ddf6b61 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 @@ -71,7 +71,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimetamp) throws SQLException; public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] tableName, PTableType tableType, Map<String,Object> tableProps, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException; - public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType) throws SQLException; + public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException; public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, List<Pair<byte[],Map<String,Object>>> families, PTable table) throws SQLException; public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException; public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException; http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 e6518c5..043c8fd 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 @@ -1192,7 +1192,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } @Override - public MetaDataMutationResult dropTable(final List<Mutation> tableMetaData, final PTableType tableType) throws SQLException { + public MetaDataMutationResult dropTable(final List<Mutation> tableMetaData, final PTableType tableType, final boolean cascade) throws SQLException { byte[][] rowKeyMetadata = new byte[3][]; SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata); byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX]; @@ -1212,6 +1212,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement builder.addTableMetadataMutations(mp.toByteString()); } builder.setTableType(tableType.getSerializedValue()); + builder.setCascade(cascade); instance.dropTable(controller, builder.build(), rpcCallback); if(controller.getFailedOn() != null) { http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 81b66b2..61c2ef8 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 @@ -200,7 +200,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple } @Override - public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType) throws SQLException { + public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException { return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 306d536..0b6a399 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 @@ -115,8 +115,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple } @Override - public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType) throws SQLException { - return getDelegate().dropTable(tabeMetaData, tableType); + public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType, boolean cascade) throws SQLException { + return getDelegate().dropTable(tabeMetaData, tableType, cascade); } @Override http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/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 a3db203..56b6604 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 @@ -1430,17 +1430,17 @@ public class MetaDataClient { public MutationState dropTable(DropTableStatement statement) throws SQLException { String schemaName = statement.getTableName().getSchemaName(); String tableName = statement.getTableName().getTableName(); - return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists()); + return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists(), statement.cascade()); } public MutationState dropIndex(DropIndexStatement statement) throws SQLException { String schemaName = statement.getTableName().getSchemaName(); String tableName = statement.getIndexName().getName(); String parentTableName = statement.getTableName().getTableName(); - return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists()); + return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false); } - private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType, boolean ifExists) throws SQLException { + private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType, boolean ifExists, boolean cascade) throws SQLException { connection.rollback(); boolean wasAutoCommit = connection.getAutoCommit(); try { @@ -1463,7 +1463,7 @@ public class MetaDataClient { hasLocalIndexTable = MetaDataUtil.hasLocalIndexTable(connection, schemaName, tableName); } - MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType); + MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade); MutationCode code = result.getMutationCode(); switch(code) { case TABLE_NOT_FOUND: http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6ea5560/phoenix-protocol/src/main/MetaDataService.proto ---------------------------------------------------------------------- diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto index 1be664b..a766674 100644 --- a/phoenix-protocol/src/main/MetaDataService.proto +++ b/phoenix-protocol/src/main/MetaDataService.proto @@ -64,6 +64,7 @@ message CreateTableRequest { message DropTableRequest { repeated bytes tableMetadataMutations = 1; required string tableType = 2; + required bool cascade = 3; } message AddColumnRequest {