You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/07/20 19:14:57 UTC

[27/50] [abbrv] phoenix git commit: PHOENIX-978: Allow views to extend parent's PK only if parent's last PK column is fixed length

PHOENIX-978: Allow views to extend parent's PK only if parent's last PK column is fixed length


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

Branch: refs/heads/calcite
Commit: 66705d5170496bed7c39bd484517722ac8d09a13
Parents: a772a4a
Author: Eli Levine <el...@apache.org>
Authored: Wed Jul 8 18:46:43 2015 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Mon Jul 13 17:36:58 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |  9 ++----
 .../java/org/apache/phoenix/end2end/ViewIT.java | 17 ++++++++++
 .../phoenix/exception/SQLExceptionCode.java     |  1 +
 .../apache/phoenix/schema/MetaDataClient.java   | 33 ++++++++++++++++++++
 4 files changed, 54 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/66705d51/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index fbaded0..0425933 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -64,7 +64,6 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Objects;
@@ -2132,15 +2131,13 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
     }
     
     @Test
-    @Ignore
-    // enable this test after https://issues.apache.org/jira/browse/PHOENIX-978 is fixed 
     public void testAddExistingViewPkColumnToBaseTableWithViews() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         try {       
             conn.createStatement().execute("CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
                     + " ID char(10) NOT NULL,"
                     + " COL1 integer NOT NULL,"
-                    + " COL2 bigint NOT NULL,"
+                    + " COL2 integer NOT NULL,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
                     + " )");
             assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, -1, "ID", "COL1", "COL2");
@@ -2169,7 +2166,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
             }
             
             // add the pk column of the view to the base table
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL PRIMARY KEY");
+            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
             assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 4, -1, "ID", "COL1", "COL2", "VIEW_COL1");
             assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 4, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
@@ -2179,7 +2176,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
             assertEquals(rs.getString(1), "view1");
             assertEquals(rs.getInt(2), 12);
             assertEquals(rs.getInt(3), 13);
-            assertEquals(rs.getInt(4), 14);
+            assertEquals(rs.getBigDecimal(4).intValue(), 14);
             assertFalse(rs.next());
 
             // query view

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66705d51/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 db38ab3..1d8af35 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
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
+import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MODIFY_VIEW_PK;
 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;
@@ -522,6 +523,22 @@ public class ViewIT extends BaseViewIT {
         assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
     }
     
+    @Test
+    public void testViewAddsPKColumnWhoseParentsLastPKIsVarLength() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 VARCHAR 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";
+        try {
+            conn.createStatement().execute(ddl);
+            fail("View cannot extend PK if parent's last PK is variable length. See https://issues.apache.org/jira/browse/PHOENIX-978.");
+        } catch (SQLException e) {
+            assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
     @Test(expected=ColumnAlreadyExistsException.class)
     public void testViewAddsClashingPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66705d51/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 b9f81fb..195450c 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,6 +189,7 @@ 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 if last PK column of parent is variable length."),
     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."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66705d51/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 9d0c1aa..56c1aa2 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
@@ -79,6 +79,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
+import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
+import static org.apache.phoenix.schema.PTableType.VIEW;
 
 import java.io.IOException;
 import java.sql.Connection;
@@ -162,6 +164,8 @@ import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.UpdateStatisticsStatement;
 import org.apache.phoenix.query.ConnectionQueryServices.Feature;
@@ -1677,6 +1681,9 @@ public class MetaDataClient {
                             .setColumnName(column.getName().getString())
                             .build().buildException();
                     }
+                    if (tableType == PTableType.VIEW && viewType != ViewType.MAPPED) {
+                        throwIfLastPKOfParentIsFixedLength(parent, schemaName, tableName, colDef);
+                    }
                     if (!pkColumns.add(column)) {
                         throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                     }
@@ -2421,6 +2428,9 @@ public class MetaDataClient {
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
                         }
+                        if (colDef != null && colDef.isPK() && table.getType() == VIEW && table.getViewType() != MAPPED) {
+                            throwIfLastPKOfParentIsFixedLength(getParentOfView(table), schemaName, tableName, colDef);
+                        }
                         PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
                         columns.add(column);
                         String pkName = null;
@@ -2956,4 +2966,27 @@ public class MetaDataClient {
         return table.getTableStats();
     }
 
+    private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
+        if (isLastPKVariableLength(parent)) { 
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
+                .setSchemaName(viewSchemaName)
+                .setTableName(viewName)
+                .setColumnName(col.getColumnDefName().getColumnName())
+                .build().buildException(); }
+    }
+    
+    private boolean isLastPKVariableLength(PTable table) {
+        List<PColumn> pkColumns = table.getPKColumns();
+        if (pkColumns.size() < 1) {
+            return false;
+        } else {
+            return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
+        }
+    }
+    
+    private PTable getParentOfView(PTable view) throws SQLException {
+        SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
+        String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
+        return connection.getMetaDataCache().getTable(new PTableKey(view.getTenantId(), parentName));
+    }
 }