You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by el...@apache.org on 2015/06/29 19:40:04 UTC

phoenix git commit: PHOENIX-978: allow views to extend parent's PK

Repository: phoenix
Updated Branches:
  refs/heads/master d02b36109 -> d6044944b


PHOENIX-978: allow views to extend parent's PK


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

Branch: refs/heads/master
Commit: d6044944b7b1f4981a897c1c7626614659a972c7
Parents: d02b361
Author: Eli Levine <el...@apache.org>
Authored: Mon Jun 29 10:37:48 2015 -0700
Committer: Eli Levine <el...@apache.org>
Committed: Mon Jun 29 10:39:17 2015 -0700

----------------------------------------------------------------------
 .../end2end/TenantSpecificTablesDDLIT.java      | 22 +----
 .../java/org/apache/phoenix/end2end/ViewIT.java | 96 ++++++++++++++++++++
 .../phoenix/exception/SQLExceptionCode.java     |  2 -
 .../apache/phoenix/schema/MetaDataClient.java   | 21 +----
 4 files changed, 101 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6044944/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 e1a1970..bf86818 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
@@ -17,9 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_DEFINE_PK_FOR_VIEW;
 import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_DROP_PK;
-import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MODIFY_VIEW_PK;
 import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static org.apache.phoenix.exception.SQLExceptionCode.TABLE_UNDEFINED;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
@@ -158,16 +156,10 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     }
     
     @Test
-    public void testTenantSpecificTableCannotDeclarePK() throws SQLException {
-        try {
+    public void testTenantSpecificTableCanDeclarePK() throws SQLException {
             createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL, "CREATE VIEW TENANT_TABLE2 ( \n" + 
                     "                tenant_col VARCHAR PRIMARY KEY) AS SELECT *\n" + 
                     "                FROM PARENT_TABLE", null, nextTimestamp());
-            fail();
-        }
-        catch (SQLException expected) {
-            assertEquals(CANNOT_DEFINE_PK_FOR_VIEW.getErrorCode(), expected.getErrorCode());
-        }
     }
     
     @Test(expected=ColumnAlreadyExistsException.class)
@@ -259,20 +251,12 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     }
     
     @Test
-    public void testMutationOfPKInTenantTablesNotAllowed() throws Exception {
+    public void testDropOfPKInTenantTablesNotAllowed() throws Exception {
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
         Connection conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
         try {
-            try {
-                conn.createStatement().execute("alter table " + TENANT_TABLE_NAME + " add new_tenant_pk char(1) primary key");
-                fail();
-            }
-            catch (SQLException expected) {
-                assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), expected.getErrorCode());
-            }
-            
-            // try removing a non-PK col
+            // try removing a PK col
             try {
                 conn.createStatement().execute("alter table " + TENANT_TABLE_NAME + " drop column id");
                 fail();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6044944/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 fb58a8f..db38ab3 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
@@ -17,8 +17,11 @@
  */
 package org.apache.phoenix.end2end;
 
+import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
+import static org.apache.phoenix.exception.SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY;
 import static org.apache.phoenix.util.TestUtil.analyzeTable;
 import static org.apache.phoenix.util.TestUtil.getAllSplits;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -32,6 +35,7 @@ import java.util.List;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.QueryUtil;
@@ -461,4 +465,96 @@ public class ViewIT extends BaseViewIT {
                 "CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER I1 [1,100] - [2,109]\n" + 
                 "    SERVER FILTER BY (\"S2\" = 'bas' AND \"S1\" = 'foo')", queryPlan);
     }
+	
+    @Test
+    public void testCreateViewDefinesPKColumn() 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, k3 VARCHAR PRIMARY KEY) AS SELECT * FROM tp WHERE K1 = 1";
+        conn.createStatement().execute(ddl);
+
+        // assert PK metadata
+        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, "V1");
+        assertPKs(rs, new String[] {"K1", "K2", "K3"});
+        
+        // sanity check upserts into base table and view
+        conn.createStatement().executeUpdate("upsert into tp (k1, k2, v1) values (1, 1, 1)");
+        conn.createStatement().executeUpdate("upsert into v1 (k1, k2, k3, v2) values (1, 1, 'abc', 'def')");
+        conn.commit();
+        
+        // expect 2 rows in the base table
+        rs = conn.createStatement().executeQuery("select count(*) from tp");
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        
+        // expect 2 row in the view
+        rs = conn.createStatement().executeQuery("select count(*) from v1");
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+    }
+    
+    @Test
+    public void testCreateViewDefinesPKConstraint() 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, k3 VARCHAR, k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4)) AS SELECT * FROM tp WHERE K1 = 1";
+        conn.createStatement().execute(ddl);
+
+        // assert PK metadata
+        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, "V1");
+        assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
+    }
+    
+    @Test
+    public void testViewAddsPKColumn() 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  AS SELECT * FROM tp WHERE v1 = 1.0";
+        conn.createStatement().execute(ddl);
+        ddl = "ALTER VIEW V1 ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
+        conn.createStatement().execute(ddl);
+
+        // assert PK metadata
+        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, "V1");
+        assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
+    }
+    
+    @Test(expected=ColumnAlreadyExistsException.class)
+    public void testViewAddsClashingPKColumn() 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  AS SELECT * FROM tp WHERE v1 = 1.0";
+        conn.createStatement().execute(ddl);
+        ddl = "ALTER VIEW V1 ADD k3 VARCHAR PRIMARY KEY, k2 VARCHAR PRIMARY KEY, v2 INTEGER";
+        conn.createStatement().execute(ddl);
+    }
+    
+    @Test
+    public void testViewAddsNotNullPKColumn() 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  AS SELECT * FROM tp WHERE v1 = 1.0";
+        conn.createStatement().execute(ddl);
+        try {
+            ddl = "ALTER VIEW V1 ADD k3 VARCHAR NOT NULL PRIMARY KEY"; 
+            conn.createStatement().execute(ddl);
+            fail("can only add nullable PKs via ALTER VIEW/TABLE");
+        } catch (SQLException e) {
+            assertEquals(NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    private void assertPKs(ResultSet rs, String[] expectedPKs) throws SQLException {
+        List<String> pkCols = newArrayListWithExpectedSize(expectedPKs.length);
+        while (rs.next()) {
+            pkCols.add(rs.getString("COLUMN_NAME"));
+        }
+        String[] actualPKs = pkCols.toArray(new String[0]);
+        assertArrayEquals(expectedPKs, actualPKs);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6044944/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index cf72384..cc8b02a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -189,7 +189,6 @@ public enum SQLExceptionCode {
     NOT_NULLABLE_COLUMN_IN_ROW_KEY(1006, "42J04", "Only nullable columns may be added to a multi-part row key."),
     VARBINARY_LAST_PK(1015, "42J04", "Cannot add column to table when the last PK column is of type VARBINARY or ARRAY."),
     NULLABLE_FIXED_WIDTH_LAST_PK(1023, "42J04", "Cannot add column to table when the last PK column is nullable and fixed width."),
-    CANNOT_MODIFY_VIEW_PK(1036, "42J04", "Cannot modify the primary key of a VIEW."),
     BASE_TABLE_COLUMN(1037, "42J04", "Cannot modify columns of base table used by tenant-specific tables."),
     // Key/value column related errors
     KEY_VALUE_NOT_NULL(1007, "42K01", "A key/value column may not be declared as not null."),
@@ -234,7 +233,6 @@ public enum SQLExceptionCode {
             
             
     CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection"),
-    CANNOT_DEFINE_PK_FOR_VIEW(1031, "42Y90", "Defining PK columns for a VIEW is not allowed."),
     DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE(1034, "42Y93", "Default column family may only be specified when creating a table."),
     INSUFFICIENT_MULTI_TENANT_COLUMNS(1040, "42Y96", "A MULTI_TENANT table must have two or more PK columns with the first column being NOT NULL and of type VARCHAR or CHAR."),
     VIEW_WHERE_IS_CONSTANT(1045, "43A02", "WHERE clause in VIEW should not evaluate to a constant."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6044944/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 e7c3cd5..d77ded8 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
@@ -1636,6 +1636,7 @@ public class MetaDataClient {
                     pkColumns.add(SaltingUtil.SALTING_COLUMN);
                 }
             }
+            int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
 
             for (ColumnDef colDef : colDefs) {
@@ -1666,13 +1667,6 @@ public class MetaDataClient {
                             .setColumnName(column.getName().getString())
                             .build().buildException();
                     }
-                    if (tableType == PTableType.VIEW && viewType != ViewType.MAPPED) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DEFINE_PK_FOR_VIEW)
-                            .setSchemaName(schemaName)
-                            .setTableName(tableName)
-                            .setColumnName(colDef.getColumnDefName().getColumnName())
-                            .build().buildException();
-                    }
                     if (!pkColumns.add(column)) {
                         throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                     }
@@ -1702,7 +1696,7 @@ public class MetaDataClient {
                     .setTableName(tableName)
                     .build().buildException();
             }
-            if (!pkColumnsNames.isEmpty() && pkColumnsNames.size() != pkColumns.size() - positionOffset) { // Then a column name in the primary key constraint wasn't resolved
+            if (!pkColumnsNames.isEmpty() && pkColumnsNames.size() != pkColumns.size() - pkPositionOffset) { // Then a column name in the primary key constraint wasn't resolved
                 Iterator<Pair<ColumnName,SortOrder>> pkColumnNamesIterator = pkColumnsNames.iterator();
                 while (pkColumnNamesIterator.hasNext()) {
                     ColumnName colName = pkColumnNamesIterator.next().getFirst();
@@ -2414,7 +2408,6 @@ public class MetaDataClient {
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
                         }
-                        throwIfAlteringViewPK(colDef, table);
                         PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
                         columns.add(column);
                         String pkName = null;
@@ -2914,16 +2907,6 @@ public class MetaDataClient {
         return result.getFunctions();
     }
 
-    private void throwIfAlteringViewPK(ColumnDef col, PTable table) throws SQLException {
-        if (col != null && col.isPK() && table.getType() == PTableType.VIEW) {
-            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-            .setSchemaName(table.getSchemaName().getString())
-            .setTableName(table.getTableName().getString())
-            .setColumnName(col.getColumnDefName().getColumnName())
-            .build().buildException();
-        }
-    }
-
     public PTableStats getTableStats(PTable table) throws SQLException {
         /*
          *  The shared view index case is tricky, because we don't have