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/09/04 09:09:51 UTC

git commit: PHOENIX-1098 Support CASCADE option on DROP TABLE that drops all VIEWs (Jan Fernando)

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 <jt...@salesforce.com>
Authored: Thu Sep 4 00:13:27 2014 -0700
Committer: James Taylor <jt...@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 {