You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/04 22:13:33 UTC

[06/50] [abbrv] phoenix git commit: PHOENIX-3414 Validate DEFAULT when used in ALTER statement

PHOENIX-3414 Validate DEFAULT when used in ALTER statement


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

Branch: refs/heads/encodecolumns2
Commit: 7f5d79adef4e6733ac29f7ed60261383ade0c6ff
Parents: 5ea0921
Author: James Taylor <ja...@apache.org>
Authored: Wed Oct 26 18:35:12 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Wed Oct 26 18:49:29 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/DefaultColumnValueIT.java   |  37 ++++++-
 .../phoenix/compile/CreateTableCompiler.java    |  60 +----------
 .../org/apache/phoenix/parse/ColumnDef.java     |  65 ++++++++++++
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +
 .../phoenix/compile/QueryCompilerTest.java      | 102 ++++++++++++++++++-
 5 files changed, 210 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f5d79ad/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
index ea9df50..783dd75 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
@@ -57,12 +57,12 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk2 INTEGER NOT NULL, " +
                 "pk3 INTEGER NOT NULL DEFAULT 10, " +
                 "test1 INTEGER, " +
-                "test2 INTEGER DEFAULT 5, " +
-                "test3 INTEGER, " +
                 "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2, pk3))";
 
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute(ddl);
+        conn.createStatement().execute("ALTER TABLE " + sharedTable1 + 
+                " ADD test2 INTEGER DEFAULT 5, est3 INTEGER");
 
         String dml = "UPSERT INTO " + sharedTable1 + " VALUES (1, 2)";
         conn.createStatement().execute(dml);
@@ -100,6 +100,39 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    public void testDefaultColumnValueOnView() throws Exception {
+        String ddl = "CREATE TABLE IF NOT EXISTS " + sharedTable1 + " (" +
+                "pk1 INTEGER NOT NULL, " +
+                "pk2 INTEGER NOT NULL, " +
+                "pk3 INTEGER NOT NULL DEFAULT 10, " +
+                "test1 INTEGER, " +
+                "test2 INTEGER DEFAULT 5, " +
+                "test3 INTEGER, " +
+                "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2, pk3))";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.createStatement().execute("CREATE VIEW " + sharedTable2 + 
+                "(pk4 INTEGER NOT NULL DEFAULT 20 PRIMARY KEY, test4 VARCHAR DEFAULT 'foo') " +
+                "AS SELECT * FROM " + sharedTable1 + " WHERE pk1 = 1");
+
+        String dml = "UPSERT INTO " + sharedTable2 + "(pk2) VALUES (2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT pk1,pk2,pk3,pk4,test2,test4 FROM " + sharedTable2);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertEquals(10, rs.getInt(3));
+        assertEquals(20, rs.getInt(4));
+        assertEquals(5, rs.getInt(5));
+        assertEquals("foo", rs.getString(6));
+        assertFalse(rs.next());
+    }
+
+    @Test
     public void testDefaultColumnValueProjected() throws Exception {
         String ddl = "CREATE TABLE IF NOT EXISTS " + sharedTable1 + " (" +
                 "pk1 INTEGER NOT NULL, " +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f5d79ad/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index 3cabfbb..07df105 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -54,8 +54,6 @@ import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.DelegateConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
-import org.apache.phoenix.schema.ConstraintViolationException;
-import org.apache.phoenix.schema.DelegateSQLException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PTable;
@@ -66,7 +64,6 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.QueryUtil;
 
 import com.google.common.collect.Iterators;
@@ -108,59 +105,12 @@ public class CreateTableCompiler {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNALLOWED_COLUMN_FAMILY)
                         .build().buildException();
             }
-            if (columnDef.getExpression() != null) {
-                ExpressionCompiler compiler = new ExpressionCompiler(context);
-                ParseNode defaultParseNode =
-                        new SQLParser(columnDef.getExpression()).parseExpression();
-                Expression defaultExpression = defaultParseNode.accept(compiler);
-                if (!defaultParseNode.isStateless()
-                        || defaultExpression.getDeterminism() != Determinism.ALWAYS) {
-                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_DEFAULT)
-                            .setColumnName(columnDef.getColumnDefName().getColumnName()).build()
-                            .buildException();
-                }
-                if (columnDef.isRowTimestamp() || ( pkConstraint != null && pkConstraint.isColumnRowTimestamp(columnDef.getColumnDefName()))) {
-                    throw new SQLExceptionInfo.Builder(
-                            SQLExceptionCode.CANNOT_CREATE_DEFAULT_ROWTIMESTAMP)
-                            .setColumnName(columnDef.getColumnDefName().getColumnName())
-                            .build().buildException();
-                }
-                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                // Evaluate the expression to confirm it's validity
-                LiteralExpression defaultValue = ExpressionUtil.getConstantExpression(defaultExpression, ptr);
-                // A DEFAULT that evaluates to null should be ignored as it adds nothing
-                if (defaultValue.getValue() == null) {
-                    if (overideColumnDefs == null) {
-                        overideColumnDefs = new ArrayList<>(columnDefs);
-                    }
-                    overideColumnDefs.set(i, new ColumnDef(columnDef, null));
-                    continue;
-                }
-                PDataType sourceType = defaultExpression.getDataType();
-                PDataType targetType = columnDef.getDataType();
-                // Ensure that coercion works (will throw if not)
-                context.getTempPtr().set(ptr.get(), ptr.getOffset(), ptr.getLength());
-                try {
-                    targetType.coerceBytes(context.getTempPtr(), defaultValue.getValue(), sourceType,
-                            defaultValue.getMaxLength(), defaultValue.getScale(),
-                            defaultValue.getSortOrder(),
-                            columnDef.getMaxLength(), columnDef.getScale(),
-                            columnDef.getSortOrder());
-                } catch (ConstraintViolationException e) {
-                    if (e.getCause() instanceof SQLException) {
-                        SQLException sqlE = (SQLException) e.getCause();
-                        throw new DelegateSQLException(sqlE, ". DEFAULT " + SQLExceptionInfo.COLUMN_NAME + "=" + columnDef.getColumnDefName().getColumnName());
-                    }
-                    throw e;
-                }
-                if (!targetType.isSizeCompatible(ptr, defaultValue.getValue(), sourceType, 
-                        defaultValue.getMaxLength(), defaultValue.getScale(), 
-                        columnDef.getMaxLength(), columnDef.getScale())) {
-                    throw new SQLExceptionInfo.Builder(
-                            SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(columnDef.getColumnDefName().getColumnName())
-                            .setMessage("DEFAULT " + columnDef.getExpression()).build()
-                            .buildException();            
+            // False means we do not need the default (because it evaluated to null)
+            if (!columnDef.validateDefault(context, pkConstraint)) {
+                if (overideColumnDefs == null) {
+                    overideColumnDefs = new ArrayList<>(columnDefs);
                 }
+                overideColumnDefs.set(i, new ColumnDef(columnDef, null));
             }
         }
         if (overideColumnDefs != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f5d79ad/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
index 6dd1dc6..4b148dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
@@ -19,12 +19,21 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.ConstraintViolationException;
+import org.apache.phoenix.schema.DelegateSQLException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.base.Preconditions;
@@ -225,4 +234,60 @@ public class ColumnDef {
         }
         return buf.toString();
     }
+    
+    public boolean validateDefault(StatementContext context, PrimaryKeyConstraint pkConstraint) throws SQLException {
+        String defaultStr = this.getExpression();
+        if (defaultStr == null) {
+            return true;
+        }
+        ExpressionCompiler compiler = new ExpressionCompiler(context);
+        ParseNode defaultParseNode =
+                new SQLParser(this.getExpression()).parseExpression();
+        Expression defaultExpression = defaultParseNode.accept(compiler);
+        if (!defaultParseNode.isStateless()
+                || defaultExpression.getDeterminism() != Determinism.ALWAYS) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_DEFAULT)
+                    .setColumnName(this.getColumnDefName().getColumnName()).build()
+                    .buildException();
+        }
+        if (this.isRowTimestamp() || ( pkConstraint != null && pkConstraint.isColumnRowTimestamp(this.getColumnDefName()))) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.CANNOT_CREATE_DEFAULT_ROWTIMESTAMP)
+                    .setColumnName(this.getColumnDefName().getColumnName())
+                    .build().buildException();
+        }
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        // Evaluate the expression to confirm it's validity
+        LiteralExpression defaultValue = ExpressionUtil.getConstantExpression(defaultExpression, ptr);
+        // A DEFAULT that evaluates to null should be ignored as it adds nothing
+        if (defaultValue.getValue() == null) {
+            return false;
+        }
+        PDataType sourceType = defaultExpression.getDataType();
+        PDataType targetType = this.getDataType();
+        // Ensure that coercion works (will throw if not)
+        context.getTempPtr().set(ptr.get(), ptr.getOffset(), ptr.getLength());
+        try {
+            targetType.coerceBytes(context.getTempPtr(), defaultValue.getValue(), sourceType,
+                    defaultValue.getMaxLength(), defaultValue.getScale(),
+                    defaultValue.getSortOrder(),
+                    this.getMaxLength(), this.getScale(),
+                    this.getSortOrder());
+        } catch (ConstraintViolationException e) {
+            if (e.getCause() instanceof SQLException) {
+                SQLException sqlE = (SQLException) e.getCause();
+                throw new DelegateSQLException(sqlE, ". DEFAULT " + SQLExceptionInfo.COLUMN_NAME + "=" + this.getColumnDefName().getColumnName());
+            }
+            throw e;
+        }
+        if (!targetType.isSizeCompatible(ptr, defaultValue.getValue(), sourceType, 
+                defaultValue.getMaxLength(), defaultValue.getScale(), 
+                this.getMaxLength(), this.getScale())) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(this.getColumnDefName().getColumnName())
+                    .setMessage("DEFAULT " + this.getExpression()).build()
+                    .buildException();            
+        }
+        return true;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f5d79ad/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 93fddae..ecd5f7a 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
@@ -2930,6 +2930,7 @@ public class MetaDataClient {
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
                 if (columnDefs.size() > 0 ) {
+                    StatementContext context = new StatementContext(new PhoenixStatement(connection), resolver);
                     try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE)) {
                         short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
                         for( ColumnDef colDef : columnDefs) {
@@ -2949,6 +2950,9 @@ public class MetaDataClient {
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_CREATE_ONLY)
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
+                            if (!colDef.validateDefault(context, null)) {
+                                colDef = new ColumnDef(colDef, null); // Remove DEFAULT as it's not necessary
+                            }
                             PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
                             columns.add(column);
                             String pkName = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f5d79ad/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index ee9d6c8..393da4c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -2484,6 +2484,23 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testAlterTableStatefulDefault() throws Exception {
+        String ddl = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY)";
+        String ddl2 = "ALTER TABLE table_with_default " +
+                "ADD datecol DATE DEFAULT CURRENT_DATE()";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        try {
+            conn.createStatement().execute(ddl2);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_CREATE_DEFAULT.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    @Test
     public void testDefaultTypeMismatch() throws Exception {
         String ddl = "CREATE TABLE table_with_default (" +
                 "pk INTEGER PRIMARY KEY, " +
@@ -2499,7 +2516,41 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
-    public void testDefaultRowTimestamp() throws Exception {
+    public void testAlterTableDefaultTypeMismatch() throws Exception {
+        String ddl = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY)";
+        String ddl2 = "ALTER TABLE table_with_default " +
+                "ADD v CHAR(3) DEFAULT 1";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        try {
+            conn.createStatement().execute(ddl2);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    @Test
+    public void testDefaultTypeMismatchInView() throws Exception {
+        String ddl1 = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY, " +
+                "v VARCHAR DEFAULT 'foo')";
+        String ddl2 = "CREATE VIEW my_view(v2 VARCHAR DEFAULT 1) AS SELECT * FROM table_with_default";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl1);
+        try {
+            conn.createStatement().execute(ddl2);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
+        }
+    }
+
+    @Test
+    public void testDefaultRowTimestamp1() throws Exception {
         String ddl = "CREATE TABLE IF NOT EXISTS table_with_defaults ("
                 + "pk1 INTEGER NOT NULL,"
                 + "pk2 BIGINT NOT NULL DEFAULT 5,"
@@ -2518,6 +2569,23 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testDefaultRowTimestamp2() throws Exception {
+        String ddl = "CREATE TABLE table_with_defaults ("
+                + "k BIGINT DEFAULT 5 PRIMARY KEY ROW_TIMESTAMP)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(
+                    SQLExceptionCode.CANNOT_CREATE_DEFAULT_ROWTIMESTAMP.getErrorCode(),
+                    e.getErrorCode());
+        }
+    }
+
+    @Test
     public void testDefaultSizeMismatch() throws Exception {
         String ddl = "CREATE TABLE table_with_default (" +
                 "pk INTEGER PRIMARY KEY, " +
@@ -2533,6 +2601,23 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testAlterTableDefaultSizeMismatch() throws Exception {
+        String ddl = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY)";
+        String ddl2 = "ALTER TABLE table_with_default " +
+                "ADD v CHAR(3) DEFAULT 'foobar'";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        try {
+            conn.createStatement().execute(ddl2);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    @Test
     public void testNullDefaultRemoved() throws Exception {
         String ddl = "CREATE TABLE table_with_default (" +
                 "pk INTEGER PRIMARY KEY, " +
@@ -2546,6 +2631,21 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testNullAlterTableDefaultRemoved() throws Exception {
+        String ddl = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY)";
+        String ddl2 = "ALTER TABLE table_with_default " +
+                "ADD v CHAR(3) DEFAULT null";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.createStatement().execute(ddl2);
+        PTable table = conn.unwrap(PhoenixConnection.class).getMetaDataCache()
+                .getTableRef(new PTableKey(null,"TABLE_WITH_DEFAULT")).getTable();
+        assertNull(table.getColumn("V").getExpressionStr());
+    }
+
+    @Test
     public void testIndexOnViewWithChildView() throws SQLException {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE TABLE PLATFORM_ENTITY.GLOBAL_TABLE (\n" +