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:31 UTC

[04/50] [abbrv] phoenix git commit: PHOENIX-476 Support declaration of DEFAULT in CREATE statement (Kevin Liew)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateSQLException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateSQLException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateSQLException.java
new file mode 100644
index 0000000..9ed4805
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateSQLException.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+package org.apache.phoenix.schema;
+
+import java.sql.SQLException;
+import java.util.Iterator;
+
+public class DelegateSQLException extends SQLException {
+    private final SQLException delegate;
+    private final String msg;
+    
+    public DelegateSQLException(SQLException e, String msg) {
+        this.delegate = e;
+        this.msg = e.getMessage() + msg;
+    }
+    
+    @Override
+    public String getMessage() {
+        return msg;
+    }
+    
+    @Override
+    public String getSQLState() {
+        return delegate.getSQLState();
+    }
+
+    @Override
+    public int getErrorCode() {
+        return delegate.getErrorCode();
+    }
+
+    @Override
+    public SQLException getNextException() {
+        return delegate.getNextException();
+    }
+
+    @Override
+    public void setNextException(SQLException ex) {
+        delegate.setNextException(ex);
+    }
+
+    @Override
+    public Iterator<Throwable> iterator() {
+        return delegate.iterator();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/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 285c8fa..93fddae 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
@@ -1370,6 +1370,9 @@ public class MetaDataClient {
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
+                        if (colRef.getColumn().getExpressionStr() != null) {
+                            expressionStr = colRef.getColumn().getExpressionStr();
+                        }
                     }
                     else {
                         // if this is an expression
@@ -1411,7 +1414,7 @@ public class MetaDataClient {
                     if (!SchemaUtil.isPKColumn(col) && col.getViewConstant() == null) {
                         // Need to re-create ColumnName, since the above one won't have the column family name
                         colName = ColumnName.caseSensitiveColumnName(isLocalIndex?IndexUtil.getLocalIndexColumnFamily(col.getFamilyName().getString()):col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
-                        columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder(), null, col.isRowTimestamp()));
+                        columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder(), col.getExpressionStr(), col.isRowTimestamp()));
                     }
                 }
 
@@ -3651,8 +3654,7 @@ public class MetaDataClient {
 
     public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQLException {
         // As we allow default namespace mapped to empty schema, so this is to reset schema in connection
-        if (useSchemaStatement.getSchemaName().equals(StringUtil.EMPTY_STRING)
-                || useSchemaStatement.getSchemaName().toUpperCase().equals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE)) {
+        if (useSchemaStatement.getSchemaName().equals(StringUtil.EMPTY_STRING)) {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 8d7161e..064137e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -238,7 +238,7 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, oldColumn.isRowTimestamp(), oldColumn.isDynamic());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 7f5efc8..773ce76 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.hbase.index.util.KeyValueBuilder.deleteQuietly;
 import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 
 import java.io.IOException;
+import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -39,12 +40,18 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
+import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
@@ -55,6 +62,7 @@ import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 import org.apache.phoenix.util.StringUtil;
@@ -583,10 +591,16 @@ public class PTableImpl implements PTable {
 
     @Override
     public int newKey(ImmutableBytesWritable key, byte[][] values) {
+        List<PColumn> columns = getPKColumns();
         int nValues = values.length;
         while (nValues > 0 && (values[nValues-1] == null || values[nValues-1].length == 0)) {
             nValues--;
         }
+        for (PColumn column : columns) {
+            if (column.getExpressionStr() != null) {
+                nValues++;
+            }
+        }
         int i = 0;
         TrustedByteArrayOutputStream os = new TrustedByteArrayOutputStream(SchemaUtil.estimateKeyLength(this));
         try {
@@ -596,11 +610,11 @@ public class PTableImpl implements PTable {
                 i++;
                 os.write(QueryConstants.SEPARATOR_BYTE_ARRAY);
             }
-            List<PColumn> columns = getPKColumns();
             int nColumns = columns.size();
             PDataType type = null;
             SortOrder sortOrder = null;
             boolean wasNull = false;
+
             while (i < nValues && i < nColumns) {
                 // Separate variable length column values in key with zero byte
                 if (type != null && !type.isFixedWidth()) {
@@ -612,7 +626,38 @@ public class PTableImpl implements PTable {
                 // This will throw if the value is null and the type doesn't allow null
                 byte[] byteValue = values[i++];
                 if (byteValue == null) {
-                    byteValue = ByteUtil.EMPTY_BYTE_ARRAY;
+                    if (column.getExpressionStr() != null) {
+                        try {
+                            String url = PhoenixRuntime.JDBC_PROTOCOL
+                                    + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR
+                                    + PhoenixRuntime.CONNECTIONLESS;
+                            PhoenixConnection conn = DriverManager.getConnection(url)
+                                    .unwrap(PhoenixConnection.class);
+                            StatementContext context =
+                                    new StatementContext(new PhoenixStatement(conn));
+
+                            ExpressionCompiler compiler = new ExpressionCompiler(context);
+                            ParseNode defaultParseNode =
+                                    new SQLParser(column.getExpressionStr()).parseExpression();
+                            Expression defaultExpression = defaultParseNode.accept(compiler);
+                            defaultExpression.evaluate(null, key);
+                            column.getDataType().coerceBytes(key, null,
+                                    defaultExpression.getDataType(),
+                                    defaultExpression.getMaxLength(), defaultExpression.getScale(),
+                                    defaultExpression.getSortOrder(),
+                                    column.getMaxLength(), column.getScale(),
+                                    column.getSortOrder());
+                            byteValue = ByteUtil.copyKeyBytesIfNecessary(key);
+                        } catch (SQLException e) { // should not be possible
+                            throw new ConstraintViolationException(name.getString() + "."
+                                    + column.getName().getString()
+                                    + " failed to compile default value expression of "
+                                    + column.getExpressionStr());
+                        }
+                    }
+                    else {
+                        byteValue = ByteUtil.EMPTY_BYTE_ARRAY;
+                    }
                 }
                 wasNull = byteValue.length == 0;
                 // An empty byte array return value means null. Do this,
@@ -814,10 +859,12 @@ public class PTableImpl implements PTable {
             boolean isNull = type.isNull(byteValue);
             if (isNull && !column.isNullable()) {
                 throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
-            } else if (isNull && PTableImpl.this.isImmutableRows()) {
+            } else if (isNull && PTableImpl.this.isImmutableRows()
+                    && column.getExpressionStr() == null) {
+                // Store nulls for immutable tables otherwise default value would be used
                 removeIfPresent(setValues, family, qualifier);
                 removeIfPresent(unsetValues, family, qualifier);
-            } else if (isNull && !getStoreNulls()) {
+            } else if (isNull && !getStoreNulls() && column.getExpressionStr() == null) {
                 removeIfPresent(setValues, family, qualifier);
                 deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
                             .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index 9aa3f42..7b4aa38 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -37,6 +37,16 @@ public class PBinary extends PBinaryBase {
   }
 
   @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object o, PDataType actualType, Integer actualMaxLength,
+          Integer actualScale, SortOrder actualModifier, Integer desiredMaxLength, Integer desiredScale,
+          SortOrder expectedModifier) {
+    PVarbinary.INSTANCE.coerceBytes(ptr, o, actualType, actualMaxLength, actualScale, actualModifier, desiredMaxLength, desiredScale, expectedModifier);
+    if (null != desiredMaxLength && null != expectedModifier) {
+      pad(ptr, desiredMaxLength, expectedModifier);
+    }
+  }
+
+  @Override
   public byte[] pad(byte[] b, Integer maxLength, SortOrder sortOrder) {
       if (b == null || b.length >= maxLength) {
           return b;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/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 2439ac9..ee9d6c8 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
@@ -68,6 +68,7 @@ import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDecimal;
@@ -2468,6 +2469,83 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testStatefulDefault() throws Exception {
+        String ddl = "CREATE TABLE table_with_default (" +
+                "pk INTEGER PRIMARY KEY, " +
+                "datecol DATE DEFAULT CURRENT_DATE())";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute(ddl);
+            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, " +
+                "v VARCHAR DEFAULT 1)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
+        }
+    }
+
+    @Test
+    public void testDefaultRowTimestamp() throws Exception {
+        String ddl = "CREATE TABLE IF NOT EXISTS table_with_defaults ("
+                + "pk1 INTEGER NOT NULL,"
+                + "pk2 BIGINT NOT NULL DEFAULT 5,"
+                + "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2 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, " +
+                "v CHAR(3) DEFAULT 'foobar')";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute(ddl);
+            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, " +
+                "v VARCHAR DEFAULT null)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        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" + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5ea09210/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
index 2f1a369..c0bff8a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
@@ -118,7 +118,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-        ColumnExpression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumn("ID").getPosition()).newColumnExpression();
+        Expression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumn("ID").getPosition()).newColumnExpression();
         Expression id = new RowKeyColumnExpression(idExpression,new RowKeyValueAccessor(plan.getTableRef().getTable().getPKColumns(),0));
         Expression company = new KeyValueColumnExpression(plan.getTableRef().getTable().getColumn("COMPANY"));
         // FilterList has no equals implementation