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 2016/10/26 23:35:00 UTC

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

Repository: phoenix
Updated Branches:
  refs/heads/master 202b8eb1e -> 5744c6f0e


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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/5744c6f0/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/5744c6f0/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/5744c6f0/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 7315ad6..44fc47d 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


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

Posted by ja...@apache.org.
PHOENIX-476 Support declaration of DEFAULT in CREATE statement (Kevin Liew)


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

Branch: refs/heads/master
Commit: 5744c6f0eb7119e20122a24e93c0a0c6e98e9662
Parents: 202b8eb
Author: James Taylor <ja...@apache.org>
Authored: Wed Oct 26 15:27:33 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Wed Oct 26 16:34:04 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/DefaultColumnValueIT.java   | 1037 ++++++++++++++++++
 .../phoenix/iterate/MockResultIterator.java     |    2 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |    8 +-
 .../phoenix/compile/CreateTableCompiler.java    |   78 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |    2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |    8 +-
 .../UngroupedAggregateRegionObserver.java       |    2 +-
 .../phoenix/exception/SQLExceptionCode.java     |    3 +-
 .../apache/phoenix/execute/TupleProjector.java  |    3 +-
 .../phoenix/expression/ExpressionType.java      |    4 +-
 .../function/DefaultValueExpression.java        |   91 ++
 .../org/apache/phoenix/parse/ColumnDef.java     |   14 +
 .../phoenix/parse/CreateSchemaStatement.java    |    4 +-
 .../phoenix/parse/CreateTableStatement.java     |   13 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |   14 +-
 .../phoenix/parse/UseSchemaStatement.java       |    4 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |   42 +-
 .../phoenix/schema/DelegateSQLException.java    |   62 ++
 .../apache/phoenix/schema/MetaDataClient.java   |    8 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    2 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   55 +-
 .../apache/phoenix/schema/types/PBinary.java    |   10 +
 .../phoenix/compile/QueryCompilerTest.java      |   78 ++
 .../phoenix/compile/WhereCompilerTest.java      |    2 +-
 24 files changed, 1510 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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
new file mode 100644
index 0000000..ea9df50
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
@@ -0,0 +1,1037 @@
+/*
+ * 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.end2end;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.DateUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
+    private String sharedTable1;
+    private String sharedTable2;
+
+    @Before
+    public void init() {
+        sharedTable1 = generateUniqueName();
+        sharedTable2 = generateUniqueName();
+    }
+
+    @Test
+    public void testDefaultColumnValue() 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);
+
+        String dml = "UPSERT INTO " + sharedTable1 + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + sharedTable1 + " VALUES (11, 12, 13, 14, null, 16)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+
+        String projection = "*";
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertEquals(10, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertEquals(5, rs.getInt(5));
+        assertEquals(0, rs.getInt(6));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+
+        rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals(13, rs.getInt(3));
+        assertEquals(14, rs.getInt(4));
+        assertEquals(0, rs.getInt(5));
+        assertTrue(rs.wasNull());
+        assertEquals(16, rs.getInt(6));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultColumnValueProjected() 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);
+
+        String dml = "UPSERT INTO " + sharedTable1 + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + sharedTable1 + " VALUES (11, 12, 13, 14, null, 16)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        String projection = "pk1, pk2, pk3, test1, test2, test3";
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertEquals(10, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertEquals(5, rs.getInt(5));
+        assertEquals(0, rs.getInt(6));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+
+        rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals(13, rs.getInt(3));
+        assertEquals(14, rs.getInt(4));
+        assertEquals(0, rs.getInt(5));
+        assertTrue(rs.wasNull());
+        assertEquals(16, rs.getInt(6));
+        assertFalse(rs.next());
+
+        projection = "pk1, pk3, pk2, test1, test3, test2";
+
+        rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(10, rs.getInt(2));
+        assertEquals(2, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertEquals(0, rs.getInt(5));
+        assertTrue(rs.wasNull());
+        assertEquals(5, rs.getInt(6));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement()
+                .executeQuery("SELECT " + projection + " FROM " + sharedTable1 + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(13, rs.getInt(2));
+        assertEquals(12, rs.getInt(3));
+        assertEquals(14, rs.getInt(4));
+        assertEquals(16, rs.getInt(5));
+        assertEquals(0, rs.getInt(6));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testMultipleDefaults() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " (" +
+                "pk1 INTEGER NOT NULL, " +
+                "pk2 INTEGER NOT NULL DEFAULT 5, " +
+                "pk3 INTEGER NOT NULL DEFAULT 10, " +
+                "test1 INTEGER, " +
+                "test2 INTEGER DEFAULT 50, " +
+                "test3 INTEGER DEFAULT 100, " +
+                "test4 INTEGER, " +
+                "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2, pk3))";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + table + " VALUES (11, 12, 13, 21, null, null, 24)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(5, rs.getInt(2));
+        assertEquals(10, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertEquals(50, rs.getInt(5));
+        assertEquals(100, rs.getInt(6));
+        assertEquals(0, rs.getInt(7));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + table + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals(13, rs.getInt(3));
+        assertEquals(21, rs.getInt(4));
+        assertEquals(0, rs.getInt(5));
+        assertTrue(rs.wasNull());
+        assertEquals(0, rs.getInt(6));
+        assertTrue(rs.wasNull());
+        assertEquals(24, rs.getInt(7));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultImmutableRows() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " (" +
+                "pk1 INTEGER NOT NULL, " +
+                "pk2 INTEGER NOT NULL DEFAULT 5, " +
+                "pk3 INTEGER NOT NULL DEFAULT 10, " +
+                "test1 INTEGER, " +
+                "test2 INTEGER DEFAULT 50, " +
+                "test3 INTEGER DEFAULT 100, " +
+                "test4 INTEGER, " +
+                "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2, pk3))"
+                + "IMMUTABLE_ROWS=true";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + table + " VALUES (11, 12, 13, 21, null, null, 24)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(5, rs.getInt(2));
+        assertEquals(10, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertEquals(50, rs.getInt(5));
+        assertEquals(100, rs.getInt(6));
+        assertEquals(0, rs.getInt(7));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + table + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals(13, rs.getInt(3));
+        assertEquals(21, rs.getInt(4));
+        assertEquals(0, rs.getInt(5));
+        assertTrue(rs.wasNull());
+        assertEquals(0, rs.getInt(6));
+        assertTrue(rs.wasNull());
+        assertEquals(24, rs.getInt(7));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testTrailingNullOverwritingDefault() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE " + table + " (" +
+                "pk INTEGER PRIMARY KEY, " +
+                "mid INTEGER, " +
+                "def INTEGER DEFAULT 10)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1, 10, null)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(10, rs.getInt(2));
+        assertEquals(0, rs.getInt(3));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultReinit() 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);
+
+        String dml = "UPSERT INTO " + sharedTable1 + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + sharedTable1 + " VALUES (11, 12, 13, 14, null, 16)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT pk3, test2 FROM " + sharedTable1 + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(10, rs.getInt(1));
+        assertEquals(5, rs.getInt(2));
+        assertFalse(rs.next());
+
+        conn.close();
+        Connection conn2 = DriverManager.getConnection(getUrl());
+
+        rs = conn2.createStatement()
+                .executeQuery("SELECT pk3, test2 FROM " + sharedTable1 + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(10, rs.getInt(1));
+        assertEquals(5, rs.getInt(2));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultMiddlePrimaryKey() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " (" +
+                "pk1 INTEGER NOT NULL, " +
+                "pk2 INTEGER NOT NULL DEFAULT 100, " +
+                "pk3 INTEGER NOT NULL, " +
+                "test1 INTEGER, " +
+                "CONSTRAINT NAME_PK PRIMARY KEY (pk1, pk2, pk3))";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        try {
+            conn.createStatement().execute(dml);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+
+        dml = "UPSERT INTO " + table + " VALUES (1, 2)";
+        try {
+            conn.createStatement().execute(dml);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+
+        dml = "UPSERT INTO " + table + " VALUES (1, 2, 3)";
+        conn.createStatement().execute(dml);
+
+        dml = "UPSERT INTO " + table + " (pk1, pk3) VALUES (11, 13)";
+        conn.createStatement().execute(dml);
+
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk1 = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertEquals(3, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + table + " WHERE pk1 = 11");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(100, rs.getInt(2));
+        assertEquals(13, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultMiddleKeyValueCol() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER,"
+                + "c2 INTEGER DEFAULT 50,"
+                + "c3 INTEGER)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + table + " (pk, c3) VALUES (10, 100)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(0, rs.getInt(2));
+        assertTrue(rs.wasNull());
+        assertEquals(50, rs.getInt(3));
+        assertEquals(0, rs.getInt(4));
+        assertTrue(rs.wasNull());
+
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + table + " WHERE pk = 10");
+        assertTrue(rs.next());
+        assertEquals(10, rs.getInt(1));
+        assertEquals(0, rs.getInt(2));
+        assertTrue(rs.wasNull());
+        assertEquals(50, rs.getInt(3));
+        assertEquals(100, rs.getInt(4));
+    }
+
+    @Test
+    public void testDefaultAllDataTypesKeyValueCol() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER PRIMARY KEY,"
+                + "int INTEGER DEFAULT -100,"
+                + "uint UNSIGNED_INT DEFAULT 100, "
+                + "bint BIGINT DEFAULT -200,"
+                + "ubint UNSIGNED_LONG DEFAULT 200,"
+                + "tint TINYINT DEFAULT -50,"
+                + "utint UNSIGNED_TINYINT DEFAULT 50,"
+                + "sint SMALLINT DEFAULT -10,"
+                + "usint UNSIGNED_SMALLINT DEFAULT 10,"
+                + "flo FLOAT DEFAULT -100.8,"
+                + "uflo UNSIGNED_FLOAT DEFAULT 100.9,"
+                + "doub DOUBLE DEFAULT -200.5,"
+                + "udoubl UNSIGNED_DOUBLE DEFAULT 200.8,"
+                + "dec DECIMAL DEFAULT -654624562.3462642362,"
+                + "bool BOOLEAN DEFAULT true,"
+                + "tim TIME DEFAULT time '1900-10-01 14:03:22.559',"
+                + "dat DATE DEFAULT date '1900-10-01 14:03:22.559',"
+                + "timest TIMESTAMP DEFAULT timestamp '1900-10-01 14:03:22.559',"
+                + "utim UNSIGNED_TIME DEFAULT time '2005-10-01 14:03:22.559',"
+                + "udat UNSIGNED_DATE DEFAULT date '2005-10-01 14:03:22.559',"
+                + "utimest UNSIGNED_TIMESTAMP DEFAULT timestamp '2005-10-01 14:03:22.559',"
+                + "vc VARCHAR DEFAULT 'ABCD',"
+                + "c CHAR(5) DEFAULT 'EF',"
+                + "bin BINARY(5) DEFAULT 'MNOP',"
+                + "varbin VARBINARY DEFAULT 'QR'"
+                + ")";
+
+        testDefaultAllDataTypes(table, ddl);
+    }
+
+    @Test
+    public void testDefaultAllDataTypesPrimaryKey() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER NOT NULL,"
+                + "int INTEGER NOT NULL DEFAULT -100,"
+                + "uint UNSIGNED_INT NOT NULL DEFAULT 100, "
+                + "bint BIGINT NOT NULL DEFAULT -200,"
+                + "ubint UNSIGNED_LONG NOT NULL DEFAULT 200,"
+                + "tint TINYINT NOT NULL DEFAULT -50,"
+                + "utint UNSIGNED_TINYINT NOT NULL DEFAULT 50,"
+                + "sint SMALLINT NOT NULL DEFAULT -10,"
+                + "usint UNSIGNED_SMALLINT NOT NULL DEFAULT 10,"
+                + "flo FLOAT NOT NULL DEFAULT -100.8,"
+                + "uflo UNSIGNED_FLOAT NOT NULL DEFAULT 100.9,"
+                + "doub DOUBLE NOT NULL DEFAULT -200.5,"
+                + "udoub UNSIGNED_DOUBLE NOT NULL DEFAULT 200.8,"
+                + "dec DECIMAL NOT NULL DEFAULT -654624562.3462642362,"
+                + "bool BOOLEAN NOT NULL DEFAULT true,"
+                + "tim TIME NOT NULL DEFAULT time '1900-10-01 14:03:22.559',"
+                + "dat DATE NOT NULL DEFAULT date '1900-10-01 14:03:22.559',"
+                + "timest TIMESTAMP NOT NULL DEFAULT timestamp '1900-10-01 14:03:22.559',"
+                + "utim UNSIGNED_TIME NOT NULL DEFAULT time '2005-10-01 14:03:22.559',"
+                + "udat UNSIGNED_DATE NOT NULL DEFAULT date '2005-10-01 14:03:22.559',"
+                + "utimest UNSIGNED_TIMESTAMP NOT NULL DEFAULT timestamp '2005-10-01 14:03:22.559',"
+                + "vc VARCHAR NOT NULL DEFAULT 'ABCD',"
+                + "c CHAR(5) NOT NULL DEFAULT 'EF',"
+                + "bin BINARY(5) NOT NULL DEFAULT 'MNOP',"
+                + "varbin VARBINARY NOT NULL DEFAULT 'QR'"
+                + "CONSTRAINT pk_final PRIMARY KEY (pk, int, uint, bint, ubint, tint, utint,"
+                + "sint, usint, flo, uflo, doub, udoub, dec, bool,"
+                + "tim, dat, timest, utim, udat, utimest,"
+                + "vc, c, bin, varbin)"
+                + ")";
+
+        testDefaultAllDataTypes(table, ddl);
+    }
+
+    private void testDefaultAllDataTypes(String table, String ddl) throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(-100, rs.getInt(2));
+        assertEquals(100, rs.getInt(3));
+        assertEquals(-200, rs.getLong(4));
+        assertEquals(200, rs.getLong(5));
+        assertEquals(-50, rs.getByte(6));
+        assertEquals(50, rs.getByte(7));
+        assertEquals(-10, rs.getShort(8));
+        assertEquals(10, rs.getShort(9));
+        assertEquals(new Float(-100.8), rs.getFloat(10), 0);
+        assertEquals(new Float(100.9), rs.getFloat(11), 0);
+        assertEquals(-200.5, rs.getDouble(12), 0);
+        assertEquals(200.8, rs.getDouble(13), 0);
+        assertEquals(new BigDecimal("-654624562.3462642362"), rs.getBigDecimal(14));
+        assertEquals(true, rs.getBoolean(15));
+        assertEquals(DateUtil.parseTime("1900-10-01 14:03:22.559"), rs.getTime(16));
+        assertEquals(DateUtil.parseDate("1900-10-01 14:03:22.559"), rs.getDate(17));
+        assertEquals(DateUtil.parseTimestamp("1900-10-01 14:03:22.559"), rs.getTimestamp(18));
+        assertEquals(DateUtil.parseTime("2005-10-01 14:03:22.559"), rs.getTime(19));
+        assertEquals(DateUtil.parseDate("2005-10-01 14:03:22.559"), rs.getDate(20));
+        assertEquals(DateUtil.parseTimestamp("2005-10-01 14:03:22.559"), rs.getTimestamp(21));
+        assertEquals("ABCD", rs.getString(22));
+        assertEquals("EF", rs.getString(23));
+        assertArrayEquals(
+                ByteUtil.fillKey(new byte[] {'M', 'N', 'O', 'P'}, rs.getBytes(24).length),
+                rs.getBytes(24));
+        assertArrayEquals(new byte[] {'Q', 'R'}, rs.getBytes(25));
+    }
+
+    @Test
+    public void testDefaultExpression() throws Exception {
+        String ddl = "CREATE TABLE IF NOT EXISTS " + sharedTable2 + " (" +
+                "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER DEFAULT 1 + 9,"
+                + "c2 DOUBLE DEFAULT SQRT(91506.25),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
+                + "c5 CHAR(5) DEFAULT 'E' || 'F',"
+                + "c6 INTEGER DEFAULT MONTH(TO_TIMESTAMP('2015-6-05'))"
+                + ")";
+
+        verifyDefaultExpression(sharedTable2, ddl);
+    }
+
+    @Test
+    public void testDefaultExpressionPrimaryKey() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " (" +
+                "pk INTEGER NOT NULL,"
+                + "c1 INTEGER NOT NULL DEFAULT 1 + 9,"
+                + "c2 DOUBLE NOT NULL DEFAULT SQRT(91506.25),"
+                + "c3 DECIMAL NOT NULL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c4 VARCHAR NOT NULL DEFAULT 'AB' || 'CD',"
+                + "c5 CHAR(5) NOT NULL DEFAULT 'E' || 'F',"
+                + "c6 INTEGER NOT NULL DEFAULT MONTH(TO_TIMESTAMP('2015-6-05')),"
+                + "CONSTRAINT pk_key PRIMARY KEY (pk,c1,c2,c3,c4,c5,c6)"
+                + ")";
+
+        verifyDefaultExpression(table, ddl);
+    }
+
+    private void verifyDefaultExpression(String table, String ddl) throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(10, rs.getInt(2));
+        assertEquals(302.5, rs.getDouble(3), 0);
+        assertEquals(new BigDecimal("123.33"), rs.getBigDecimal(4));
+        assertEquals("ABCD", rs.getString(5));
+        assertEquals("EF", rs.getString(6));
+        assertEquals(6, rs.getInt(7));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultUpsertSelectPrimaryKey() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String selectTable = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + selectTable + " ("
+                + "pk INTEGER PRIMARY KEY)";
+        conn.createStatement().execute(ddl);
+
+        String table = generateUniqueName();
+        ddl = "CREATE TABLE IF NOT EXISTS " + table + " ("
+                + "pk1 INTEGER NOT NULL, "
+                + "pk2 INTEGER NOT NULL DEFAULT 100,"
+                + "CONSTRAINT pk_key PRIMARY KEY(pk1, pk2))";
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String dml = "UPSERT INTO " + selectTable + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        dml = "UPSERT INTO " + selectTable + " VALUES (2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        dml = "UPSERT INTO " + table + " (pk1) SELECT pk FROM " + selectTable;
+        conn.createStatement().executeUpdate(dml);
+        dml = "UPSERT INTO " + table + " SELECT pk,pk FROM " + selectTable;
+        conn.createStatement().executeUpdate(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + selectTable);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+
+        rs =conn.createStatement().executeQuery("SELECT * FROM " + table);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(1, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(100, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        assertEquals(100, rs.getInt(2));
+    }
+
+    @Test
+    public void testDefaultArrays() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER PRIMARY KEY,"
+                + "int INTEGER[5] DEFAULT ARRAY[-100, 50],"
+                + "uint UNSIGNED_INT[5] DEFAULT ARRAY[100, 50], "
+                + "bint BIGINT[5] DEFAULT ARRAY[-200, 100],"
+                + "ubint UNSIGNED_LONG[5] DEFAULT ARRAY[200, 100],"
+                + "tint TINYINT[5] DEFAULT ARRAY[-50, 25],"
+                + "utint UNSIGNED_TINYINT[5] DEFAULT ARRAY[50, 25],"
+                + "sint SMALLINT[5] DEFAULT ARRAY[-10, 5],"
+                + "usint UNSIGNED_SMALLINT[5] DEFAULT ARRAY[10, 5],"
+                + "flo FLOAT[5] DEFAULT ARRAY[-100.8, 50.4],"
+                + "uflo UNSIGNED_FLOAT[5] DEFAULT ARRAY[100.9, 50.45],"
+                + "doub DOUBLE[5] DEFAULT ARRAY[-200.5, 100.25],"
+                + "udoubl UNSIGNED_DOUBLE[5] DEFAULT ARRAY[200.8, 100.4],"
+                + "dec DECIMAL[5] DEFAULT ARRAY[-654624562.3462642362, 3462642362.654624562],"
+                + "bool BOOLEAN[5] DEFAULT ARRAY[true, false],"
+                + "tim TIME[5] DEFAULT ARRAY["
+                + "time '1900-10-01 14:03:22.559',"
+                + "time '1990-10-01 14:03:22.559'],"
+                + "dat DATE[5] DEFAULT ARRAY["
+                + "date '1900-10-01 14:03:22.559',"
+                + "date '1990-10-01 14:03:22.559'],"
+                + "timest TIMESTAMP[5] DEFAULT ARRAY["
+                + "timestamp '1900-10-01 14:03:22.559',"
+                + "timestamp '1990-10-01 14:03:22.559'],"
+                + "utim UNSIGNED_TIME[5] DEFAULT ARRAY["
+                + "time '2005-10-01 14:03:22.559',"
+                + "time '2006-10-01 14:03:22.559'],"
+                + "udat UNSIGNED_DATE[5] DEFAULT ARRAY["
+                + "date '2005-10-01 14:03:22.559',"
+                + "date '2006-10-01 14:03:22.559'],"
+                + "utimest UNSIGNED_TIMESTAMP[5] DEFAULT ARRAY["
+                + "timestamp '2005-10-01 14:03:22.559',"
+                + "timestamp '2006-10-01 14:03:22.559'],"
+                + "vc VARCHAR[5] DEFAULT ARRAY['ABCD', 'XY'],"
+                + "c CHAR(5)[5] DEFAULT ARRAY['EF', 'Z'],"
+                + "bin BINARY(5)[5] DEFAULT ARRAY ['MNOP', 'mnop']"
+                + ")";
+
+        verifyArrays(table, ddl);
+    }
+
+    @Test
+    public void testDefaultArraysPrimaryKey() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER NOT NULL,"
+                + "int INTEGER[5] DEFAULT ARRAY[-100, 50],"
+                + "uint UNSIGNED_INT[5] DEFAULT ARRAY[100, 50], "
+                + "bint BIGINT[5] DEFAULT ARRAY[-200, 100],"
+                + "ubint UNSIGNED_LONG[5] DEFAULT ARRAY[200, 100],"
+                + "tint TINYINT[5] DEFAULT ARRAY[-50, 25],"
+                + "utint UNSIGNED_TINYINT[5] DEFAULT ARRAY[50, 25],"
+                + "sint SMALLINT[5] DEFAULT ARRAY[-10, 5],"
+                + "usint UNSIGNED_SMALLINT[5] DEFAULT ARRAY[10, 5],"
+                + "flo FLOAT[5] DEFAULT ARRAY[-100.8, 50.4],"
+                + "uflo UNSIGNED_FLOAT[5] DEFAULT ARRAY[100.9, 50.45],"
+                + "doub DOUBLE[5] DEFAULT ARRAY[-200.5, 100.25],"
+                + "udoubl UNSIGNED_DOUBLE[5] DEFAULT ARRAY[200.8, 100.4],"
+                + "dec DECIMAL[5] DEFAULT ARRAY[-654624562.3462642362, 3462642362.654624562],"
+                + "bool BOOLEAN[5] DEFAULT ARRAY[true, false],"
+                + "tim TIME[5] DEFAULT ARRAY["
+                + "time '1900-10-01 14:03:22.559',"
+                + "time '1990-10-01 14:03:22.559'],"
+                + "dat DATE[5] DEFAULT ARRAY["
+                + "date '1900-10-01 14:03:22.559',"
+                + "date '1990-10-01 14:03:22.559'],"
+                + "timest TIMESTAMP[5] DEFAULT ARRAY["
+                + "timestamp '1900-10-01 14:03:22.559',"
+                + "timestamp '1990-10-01 14:03:22.559'],"
+                + "utim UNSIGNED_TIME[5] DEFAULT ARRAY["
+                + "time '2005-10-01 14:03:22.559',"
+                + "time '2006-10-01 14:03:22.559'],"
+                + "udat UNSIGNED_DATE[5] DEFAULT ARRAY["
+                + "date '2005-10-01 14:03:22.559',"
+                + "date '2006-10-01 14:03:22.559'],"
+                + "utimest UNSIGNED_TIMESTAMP[5] DEFAULT ARRAY["
+                + "timestamp '2005-10-01 14:03:22.559',"
+                + "timestamp '2006-10-01 14:03:22.559'],"
+                + "vc VARCHAR[5] DEFAULT ARRAY['ABCD', 'XY'],"
+                + "c CHAR(5)[5] DEFAULT ARRAY['EF', 'Z'],"
+                + "bin BINARY(5)[5] NOT NULL DEFAULT ARRAY ['MNOP', 'mnop'],"
+                + "CONSTRAINT pk_key PRIMARY KEY (pk, bin)"
+                + ")";
+
+        verifyArrays(table, ddl);
+    }
+
+    private void verifyArrays(String table, String ddl) throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertArrayEquals(new int[]{-100, 50}, (int[])(rs.getArray(2).getArray()));
+        assertArrayEquals(new int[]{100, 50}, (int[])(rs.getArray(3).getArray()));
+        assertArrayEquals(new long[]{-200, 100}, (long[])(rs.getArray(4).getArray()));
+        assertArrayEquals(new long[]{200, 100}, (long[])(rs.getArray(5).getArray()));
+        assertArrayEquals(new byte[]{-50, 25}, (byte[])(rs.getArray(6).getArray()));
+        assertArrayEquals(new byte[]{50, 25}, (byte[])(rs.getArray(7).getArray()));
+        assertArrayEquals(new short[]{-10, 5}, (short[])(rs.getArray(8).getArray()));
+        assertArrayEquals(new short[]{10, 5}, (short[])(rs.getArray(9).getArray()));
+        assertArrayEquals(
+                new float[]{new Float(-100.8), new Float(50.4)},
+                (float[])(rs.getArray(10).getArray()), 0);
+        assertArrayEquals(
+                new float[]{new Float(100.9), new Float(50.45)},
+                (float[])(rs.getArray(11).getArray()), 0);
+        assertArrayEquals(new double[]{-200.5, 100.25}, (double[])(rs.getArray(12).getArray()), 0);
+        assertArrayEquals(new double[]{200.8, 100.4}, (double[])(rs.getArray(13).getArray()), 0);
+        assertArrayEquals(
+                new BigDecimal[]{
+                        new BigDecimal("-654624562.3462642362"),
+                        new BigDecimal("3462642362.654624562")},
+                (BigDecimal[])(rs.getArray(14).getArray()));
+        assertArrayEquals(new boolean[]{true, false}, (boolean[])(rs.getArray(15).getArray()));
+        assertArrayEquals(
+                new Time[]{
+                        DateUtil.parseTime("1900-10-01 14:03:22.559"),
+                        DateUtil.parseTime("1990-10-01 14:03:22.559")},
+                (Time[])(rs.getArray(16).getArray()));
+        assertArrayEquals(
+                new Date[]{
+                        DateUtil.parseDate("1900-10-01 14:03:22.559"),
+                        DateUtil.parseDate("1990-10-01 14:03:22.559")},
+                (Date[])(rs.getArray(17).getArray()));
+        assertArrayEquals(
+                new Timestamp[]{
+                        DateUtil.parseTimestamp("1900-10-01 14:03:22.559"),
+                        DateUtil.parseTimestamp("1990-10-01 14:03:22.559")},
+                (Timestamp[])(rs.getArray(18).getArray()));
+        assertArrayEquals(
+                new Time[]{
+                        DateUtil.parseTime("2005-10-01 14:03:22.559"),
+                        DateUtil.parseTime("2006-10-01 14:03:22.559")},
+                (Time[])(rs.getArray(19).getArray()));
+        assertArrayEquals(
+                new Date[]{
+                        DateUtil.parseDate("2005-10-01 14:03:22.559"),
+                        DateUtil.parseDate("2006-10-01 14:03:22.559")},
+                (Date[])(rs.getArray(20).getArray()));
+        assertArrayEquals(
+                new Timestamp[]{
+                        DateUtil.parseTimestamp("2005-10-01 14:03:22.559"),
+                        DateUtil.parseTimestamp("2006-10-01 14:03:22.559")},
+                (Timestamp[])(rs.getArray(21).getArray()));
+        assertArrayEquals(new String[]{"ABCD", "XY"}, (String[])(rs.getArray(22).getArray()));
+
+        String[] expected = new String[] {"EF","Z"};
+        Array array = conn.createArrayOf("CHAR", expected);
+        assertTrue(rs.getArray(23).equals(array));
+
+        byte[][] expectedByteArray = new byte[][] {
+                ByteUtil.fillKey(new byte[] {'M', 'N', 'O', 'P'}, 5),
+                ByteUtil.fillKey(new byte[] {'m', 'n', 'o', 'p'}, 5)
+        };
+        assertArrayEquals(expectedByteArray, (byte[][])rs.getArray(24).getArray());
+    }
+
+    @Test
+    public void testDefaultArrayWithNull() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 VARCHAR[5] DEFAULT ARRAY[NULL, 'ABCD', 'XY'],"
+                + "c2 VARCHAR[5] DEFAULT ARRAY['ABCD', NULL, 'XY'],"
+                + "c3 VARCHAR[5] DEFAULT ARRAY['ABCD', 'XY', NULL]"
+                + ")";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertArrayEquals(new String[]{null, "ABCD", "XY"}, (String[])(rs.getArray(2).getArray()));
+        assertArrayEquals(new String[]{"ABCD", null, "XY"}, (String[])(rs.getArray(3).getArray()));
+        assertArrayEquals(new String[]{"ABCD", "XY", null}, (String[])(rs.getArray(4).getArray()));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultArrayWithFixedWidthNull() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + "("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER[5] DEFAULT ARRAY[NULL, 2, 3],"
+                + "c2 INTEGER[5] DEFAULT ARRAY[1, NULL, 3],"
+                + "c3 INTEGER[5] DEFAULT ARRAY[1, 2, NULL]"
+                + ")";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertArrayEquals(new int[]{0, 2, 3}, (int[])(rs.getArray(2).getArray()));
+        assertArrayEquals(new int[]{1, 0, 3}, (int[])(rs.getArray(3).getArray()));
+        assertArrayEquals(new int[]{1, 2, 0}, (int[])(rs.getArray(4).getArray()));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultNull() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE " + table + " (" +
+                "pk INTEGER PRIMARY KEY, " +
+                "def INTEGER DEFAULT NULL)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + table + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs =
+                conn.createStatement().executeQuery("SELECT * FROM " + table + " WHERE pk = 1");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(0, rs.getInt(2));
+        assertTrue(rs.wasNull());
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultCoveredColumn() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " ("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER,"
+                + "c2 INTEGER DEFAULT 100)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String idx = generateUniqueName();
+        ddl = "CREATE INDEX " + idx + " on " + table + " (c1) INCLUDE (c2)";
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String dml = "UPSERT INTO " + table + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs =
+                conn.createStatement().executeQuery("SELECT c2 FROM " + table + " WHERE c1 = 2");
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultIndexed() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " ("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER,"
+                + "c2 INTEGER DEFAULT 100)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String idx = generateUniqueName();
+        ddl = "CREATE INDEX " + idx + " on " + table + " (c2)";
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String dml = "UPSERT INTO " + table + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs =
+                conn.createStatement().executeQuery("SELECT c2 FROM " + table + " WHERE c2 = 100");
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT c2 FROM " + table + " WHERE c2 = 5");
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultLocalIndexed() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " ("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER,"
+                + "c2 INTEGER DEFAULT 100)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String idx = generateUniqueName();
+        ddl = "CREATE LOCAL INDEX " + idx + " on " + table + " (c2)";
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String dml = "UPSERT INTO " + table + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs =
+                conn.createStatement().executeQuery("SELECT c2 FROM " + table + " WHERE c2 = 100");
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT c2 FROM " + table + " WHERE c2 = 5");
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultFunctionalIndexed() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + table + " ("
+                + "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER,"
+                + "c2 INTEGER DEFAULT 100)";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String idx = generateUniqueName();
+        ddl = "CREATE INDEX " + idx + " on " + table + " (c1 + c2)";
+        conn.createStatement().execute(ddl);
+        conn.commit();
+
+        String dml = "UPSERT INTO " + table + " VALUES (1, 2)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement()
+                .executeQuery("SELECT c2 FROM " + table + " WHERE c1 + c2 = 102");
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testDefaultSelectWhere() throws Exception {
+        String ddl = "CREATE TABLE IF NOT EXISTS " + sharedTable2 + " (" +
+                "pk INTEGER PRIMARY KEY,"
+                + "c1 INTEGER DEFAULT 1 + 9,"
+                + "c2 DOUBLE DEFAULT SQRT(91506.25),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
+                + "c5 CHAR(5) DEFAULT 'E' || 'F',"
+                + "c6 INTEGER DEFAULT MONTH(TO_TIMESTAMP('2015-6-05'))"
+                + ")";
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+
+        String dml = "UPSERT INTO " + sharedTable2 + " VALUES (1)";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs =
+                conn.createStatement().executeQuery("SELECT c1 FROM " + sharedTable2 + " WHERE c1 = 10");
+        assertTrue(rs.next());
+        assertEquals(10, rs.getInt(1));
+
+        rs = conn.createStatement().executeQuery("SELECT c4 FROM " + sharedTable2 + " WHERE c4 = 'ABCD'");
+        assertTrue(rs.next());
+        assertEquals("ABCD", rs.getString(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/it/java/org/apache/phoenix/iterate/MockResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/iterate/MockResultIterator.java b/phoenix-core/src/it/java/org/apache/phoenix/iterate/MockResultIterator.java
index b26dfcf..e842317 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/iterate/MockResultIterator.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/iterate/MockResultIterator.java
@@ -40,7 +40,7 @@ public class MockResultIterator implements PeekingResultIterator {
 
     private final Tuple tuple;
 
-    public MockResultIterator(String id, PTable table) {
+    public MockResultIterator(String id, PTable table) throws SQLException {
         TupleProjector projector = new TupleProjector(table);
         List<Cell> result = new ArrayList<>();
         result.add(new KeyValue(Bytes.toBytes(id), SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, Bytes.toBytes(id)));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 7e81a3a..2821a55 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -135,6 +135,7 @@ tokens
     ONLY = 'only';
     EXECUTE = 'execute';
     UPGRADE = 'upgrade';
+    DEFAULT = 'default';
 }
 
 
@@ -441,7 +442,7 @@ create_table_node returns [CreateTableStatement ret]
    
 // Parse a create schema statement.
 create_schema_node returns [CreateSchemaStatement ret]
-    :   CREATE SCHEMA (IF NOT ex=EXISTS)? s=identifier
+    :   CREATE SCHEMA (IF NOT ex=EXISTS)? (DEFAULT | s=identifier)
         {ret = factory.createSchema(s, ex!=null); }
     ;
 
@@ -642,12 +643,13 @@ column_defs returns [List<ColumnDef> ret]
 ;
 
 column_def returns [ColumnDef ret]
-    :   c=column_name dt=identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (nn=NOT? n=NULL)? (pk=PRIMARY KEY (order=ASC|order=DESC)? rr=ROW_TIMESTAMP?)?
+    :   c=column_name dt=identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (nn=NOT? n=NULL)? (DEFAULT df=expression)? (pk=PRIMARY KEY (order=ASC|order=DESC)? rr=ROW_TIMESTAMP?)?
         { $ret = factory.columnDef(c, dt, ar != null || lsq != null, a == null ? null :  Integer.parseInt( a.getText() ), nn!=null ? Boolean.FALSE : n!=null ? Boolean.TRUE : null, 
             l == null ? null : Integer.parseInt( l.getText() ),
             s == null ? null : Integer.parseInt( s.getText() ),
             pk != null, 
             order == null ? SortOrder.getDefault() : SortOrder.fromDDLValue(order.getText()),
+            df == null ? null : df.toString(),
             rr != null); }
     ;
 
@@ -903,7 +905,7 @@ multiply_divide_modulo_expression returns [ParseNode ret]
     ;
 
 use_schema_node returns [UseSchemaStatement ret]
-	:   USE s=identifier
+	:   USE (DEFAULT | s=identifier)
         {ret = factory.useSchema(s); }
     ;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 3928f66..3cabfbb 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
@@ -18,6 +18,7 @@
 package org.apache.phoenix.compile;
 
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collections;
 import java.util.Iterator;
@@ -46,12 +47,15 @@ import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnParseNode;
 import org.apache.phoenix.parse.CreateTableStatement;
 import org.apache.phoenix.parse.ParseNode;
+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.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;
@@ -62,6 +66,7 @@ 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;
@@ -77,7 +82,7 @@ public class CreateTableCompiler {
         this.operation = operation;
     }
 
-    public MutationPlan compile(final CreateTableStatement create) throws SQLException {
+    public MutationPlan compile(CreateTableStatement create) throws SQLException {
         final PhoenixConnection connection = statement.getConnection();
         ColumnResolver resolver = FromCompiler.getResolverForCreation(create, connection);
         PTableType type = create.getTableType();
@@ -93,13 +98,76 @@ public class CreateTableCompiler {
         BitSet isViewColumnReferencedToBe = null;
         // Check whether column families having local index column family suffix or not if present
         // don't allow creating table.
-        for(ColumnDef columnDef: create.getColumnDefs()) {
+        // Also validate the default values expressions.
+        List<ColumnDef> columnDefs = create.getColumnDefs();
+        List<ColumnDef> overideColumnDefs = null;
+        PrimaryKeyConstraint pkConstraint = create.getPrimaryKeyConstraint();
+        for (int i = 0; i < columnDefs.size(); i++) {
+            ColumnDef columnDef = columnDefs.get(i);
             if(columnDef.getColumnDefName().getFamilyName()!=null && columnDef.getColumnDefName().getFamilyName().contains(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNALLOWED_COLUMN_FAMILY)
-                .build().buildException();
+                        .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();            
+                }
             }
         }
-
+        if (overideColumnDefs != null) {
+            create = new CreateTableStatement (create,overideColumnDefs);
+        }
+        final CreateTableStatement finalCreate = create;
+        
         if (type == PTableType.VIEW) {
             TableRef tableRef = resolver.getTables().get(0);
             int nColumns = tableRef.getTable().getColumns().size();
@@ -190,7 +258,7 @@ public class CreateTableCompiler {
             @Override
             public MutationState execute() throws SQLException {
                 try {
-                    return client.createTable(create, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced);
+                    return client.createTable(finalCreate, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced);
                 } finally {
                     if (client.getConnection() != connection) {
                         client.getConnection().close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 3f9e6b2..1caf7be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -457,7 +457,7 @@ public class UpsertCompiler {
                     for (i = posOffset; i < table.getPKColumns().size(); i++) {
                         PColumn pkCol = table.getPKColumns().get(i);
                         if (!pkColumnsSet.get(i)) {
-                            if (!pkCol.isNullable()) {
+                            if (!pkCol.isNullable() && pkCol.getExpressionStr() == null) {
                                 throw new ConstraintViolationException(table.getName().getString() + "." + pkCol.getName().getString() + " may not be null");
                             }
                         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 65da298..85b6c57 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
@@ -2419,14 +2419,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     
     private class ColumnFinder extends StatelessTraverseAllExpressionVisitor<Void> {
         private boolean columnFound;
-        private final ColumnExpression columnExpression;
+        private final Expression columnExpression;
 
-        public ColumnFinder(ColumnExpression columnExpression) {
+        public ColumnFinder(Expression columnExpression) {
             this.columnExpression = columnExpression;
             columnFound = false;
         }
 
-        private Void process(ColumnExpression expression) {
+        private Void process(Expression expression) {
             if (expression.equals(columnExpression)) {
                 columnFound = true;
             }
@@ -2528,7 +2528,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     ColumnResolver columnResolver = FromCompiler.getResolver(baseTableRef);
                     StatementContext context = new StatementContext(statement, columnResolver);
                     Expression whereExpression = WhereCompiler.compile(context, viewWhere);
-                    ColumnExpression colExpression =
+                    Expression colExpression =
                             new ColumnRef(baseTableRef, existingViewColumn.getPosition())
                                     .newColumnExpression();
                     ColumnFinder columnFinder = new ColumnFinder(colExpression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 38f7253..1912ff5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -250,7 +250,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     }
 
     @Override
-    protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException {
+    protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException, SQLException {
         RegionCoprocessorEnvironment env = c.getEnvironment();
         Region region = env.getRegion();
         long ts = scan.getTimeRange().getMax();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 e6a26ae..2346224 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
@@ -245,7 +245,8 @@ public enum SQLExceptionCode {
     INVALID_MUTABLE_INDEX_CONFIG(1029, "42Y88", "Mutable secondary indexes must have the "
             + IndexManagementUtil.WAL_EDIT_CODEC_CLASS_KEY + " property set to "
             +  IndexManagementUtil.INDEX_WAL_EDIT_CODEC_CLASS_NAME + " in the hbase-sites.xml of every region server."),
-
+    CANNOT_CREATE_DEFAULT(1031, "42Y90", "Cannot create column with a stateful default value."),
+    CANNOT_CREATE_DEFAULT_ROWTIMESTAMP(1032, "42Y90", "Cannot create ROW_TIMESTAMP column with a default value."),
 
     CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection."),
     DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE(1034, "42Y93", "Default column family may only be specified when creating a table."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
index a884949..592b68e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
@@ -22,6 +22,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.List;
 
@@ -84,7 +85,7 @@ public class TupleProjector {
         valueSet = ValueBitSet.newInstance(schema);
     }
     
-    public TupleProjector(PTable projectedTable) {
+    public TupleProjector(PTable projectedTable) throws SQLException {
         Preconditions.checkArgument(projectedTable.getType() == PTableType.PROJECTED);
     	List<PColumn> columns = projectedTable.getColumns();
     	this.expressions = new Expression[columns.size() - projectedTable.getPKColumns().size()];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 94120e9..aa08adf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -46,6 +46,7 @@ import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
 import org.apache.phoenix.expression.function.DayOfMonthFunction;
 import org.apache.phoenix.expression.function.DecodeFunction;
+import org.apache.phoenix.expression.function.DefaultValueExpression;
 import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
 import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
 import org.apache.phoenix.expression.function.EncodeFunction;
@@ -285,7 +286,8 @@ public enum ExpressionType {
     CeilMonthExpression(CeilMonthExpression.class),
     CeilYearExpression(CeilYearExpression.class),
     DayOfWeekFunction(DayOfWeekFunction.class),
-    DayOfYearFunction(DayOfYearFunction.class);
+    DayOfYearFunction(DayOfYearFunction.class),
+    DefaultValueExpression(DefaultValueExpression.class);
 
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DefaultValueExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DefaultValueExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DefaultValueExpression.java
new file mode 100644
index 0000000..fceb442
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DefaultValueExpression.java
@@ -0,0 +1,91 @@
+/*
+ * 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.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+
+
+/**
+ *
+ * Internal function used to get the default value for a column not specified in UPSERT.
+ * If expr1 is evaluated (can be null), then it is returned, otherwise expr2 is returned.
+ *
+ */
+public class DefaultValueExpression extends ScalarFunction {
+    public static final String NAME = "DEFAULT";
+
+    public DefaultValueExpression() {
+    }
+
+    public DefaultValueExpression(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        boolean evaluated = children.get(0).evaluate(tuple, ptr);
+        if (evaluated) {
+            // Will potentially evaluate to null without evaluating the second expression
+            return true;
+        }
+        if (tuple.isImmutable()) {// True for the last time an evaluation is happening on the row
+            Expression secondChild = children.get(1);
+            if (secondChild.evaluate(tuple, ptr)) {
+                // Coerce the type of the second child to the type of the first child
+                getDataType().coerceBytes(ptr, null, secondChild.getDataType(),
+                        secondChild.getMaxLength(), secondChild.getScale(),
+                        secondChild.getSortOrder(),
+                        getMaxLength(), getScale(),
+                        getSortOrder());
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return children.get(0).getDataType();
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return children.get(0).getMaxLength();
+    }
+
+    @Override
+    public boolean isNullable() {
+        return children.get(0).isNullable() && children.get(1).isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean requiresFinalEvaluation() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 401d57b..6dd1dc6 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
@@ -50,6 +50,20 @@ public class ColumnDef {
     private final String expressionStr;
     private final boolean isRowTimestamp;
 
+    public ColumnDef(ColumnDef def, String expressionStr) {
+        this.columnDefName = def.columnDefName;
+        this.dataType = def.dataType;
+        this.isNull = def.isNull;
+        this.maxLength = def.maxLength;
+        this.scale = def.scale;
+        this.isPK = def.isPK;
+        this.sortOrder = def.sortOrder;
+        this.isArray = def.isArray;
+        this.arrSize = def.arrSize;
+        this.isRowTimestamp = def.isRowTimestamp;
+        this.expressionStr = expressionStr;
+    }
+
     ColumnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, Boolean isNull, Integer maxLength,
             Integer scale, boolean isPK, SortOrder sortOrder, String expressionStr, boolean isRowTimestamp) {
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
index 0e5e578..7c255cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
@@ -17,12 +17,14 @@
  */
 package org.apache.phoenix.parse;
 
+import org.apache.phoenix.util.SchemaUtil;
+
 public class CreateSchemaStatement extends MutableStatement {
 	private final String schemaName;
 	private final boolean ifNotExists;
 	
 	public CreateSchemaStatement(String schemaName,boolean ifNotExists) {
-		this.schemaName=schemaName;
+		this.schemaName = null == schemaName ? SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE : schemaName;
 		this.ifNotExists = ifNotExists;
 	}
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
index 48c6874..3c84cd0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
@@ -40,6 +40,19 @@ public class CreateTableStatement extends MutableStatement {
     private final TableName baseTableName;
     private final ParseNode whereClause;
     
+    public CreateTableStatement(CreateTableStatement createTable, List<ColumnDef> columns) {
+        this.tableName = createTable.tableName;
+        this.tableType = createTable.tableType;
+        this.columns = ImmutableList.copyOf(columns);
+        this.pkConstraint = createTable.pkConstraint;
+        this.splitNodes = createTable.splitNodes;
+        this.bindCount = createTable.bindCount;
+        this.props = createTable.props;
+        this.ifNotExists = createTable.ifNotExists;
+        this.baseTableName = createTable.baseTableName;
+        this.whereClause = createTable.whereClause;
+    }
+    
     protected CreateTableStatement(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint,
             List<ParseNode> splitNodes, PTableType tableType, boolean ifNotExists, 
             TableName baseTableName, ParseNode whereClause, int bindCount) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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 332ff15..6b58bed 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
@@ -266,8 +266,18 @@ public class ParseNodeFactory {
         return new ColumnDef(columnDefName, sqlTypeName, isNull, maxLength, scale, isPK, sortOrder, expressionStr, isRowTimestamp);
     }
 
-    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, Boolean isNull, Integer maxLength, Integer scale, boolean isPK, 
-        	SortOrder sortOrder, boolean isRowTimestamp) {
+    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName,
+            boolean isArray, Integer arrSize, Boolean isNull,
+            Integer maxLength, Integer scale, boolean isPK,
+        	SortOrder sortOrder, String expressionStr, boolean isRowTimestamp) {
+        return new ColumnDef(columnDefName, sqlTypeName,
+                isArray, arrSize, isNull,
+                maxLength, scale, isPK,
+                sortOrder, expressionStr, isRowTimestamp);
+    }
+
+    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, Boolean isNull, Integer maxLength, Integer scale, boolean isPK,
+            SortOrder sortOrder, boolean isRowTimestamp) {
         return new ColumnDef(columnDefName, sqlTypeName, isArray, arrSize, isNull, maxLength, scale, isPK, sortOrder, null, isRowTimestamp);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/parse/UseSchemaStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UseSchemaStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UseSchemaStatement.java
index 612e084..abba309 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/UseSchemaStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UseSchemaStatement.java
@@ -17,11 +17,13 @@
  */
 package org.apache.phoenix.parse;
 
+import org.apache.phoenix.util.StringUtil;
+
 public class UseSchemaStatement extends MutableStatement {
     private final String schemaName;
 
     public UseSchemaStatement(String schemaName) {
-        this.schemaName = schemaName;
+        this.schemaName = schemaName == null ? StringUtil.EMPTY_STRING : schemaName;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
index 76f6218..1d772b4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -17,13 +17,27 @@
  */
 package org.apache.phoenix.schema;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.http.annotation.Immutable;
-import org.apache.phoenix.expression.ColumnExpression;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.function.DefaultValueExpression;
+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.util.ExpressionUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+
 
 /**
  * 
@@ -89,12 +103,12 @@ public class ColumnRef {
         if (!tableRef.equals(other.tableRef)) return false;
         return true;
     }
-    
-    public ColumnExpression newColumnExpression() {
+
+    public Expression newColumnExpression() throws SQLException {
         return newColumnExpression(false, false);
     }
 
-    public ColumnExpression newColumnExpression(boolean schemaNameCaseSensitive, boolean colNameCaseSensitive) {
+    public Expression newColumnExpression(boolean schemaNameCaseSensitive, boolean colNameCaseSensitive) throws SQLException {
         PTable table = tableRef.getTable();
         PColumn column = this.getColumn();
         String displayName = tableRef.getColumnDisplayName(this, schemaNameCaseSensitive, colNameCaseSensitive);
@@ -108,8 +122,26 @@ public class ColumnRef {
         if (table.getType() == PTableType.PROJECTED || table.getType() == PTableType.SUBQUERY) {
         	return new ProjectedColumnExpression(column, table, displayName);
         }
+
+        Expression expression = new KeyValueColumnExpression(column, displayName);
+
+        if (column.getExpressionStr() != null) {
+            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);
+            if (!ExpressionUtil.isNull(defaultExpression, new ImmutableBytesWritable())) {
+                return new DefaultValueExpression(Arrays.asList(expression, defaultExpression));
+            }
+        }
        
-        return new KeyValueColumnExpression(column, displayName);
+        return expression;
     }
 
     public ColumnRef cloneAtTimestamp(long timestamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5744c6f0/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/5744c6f0/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/5744c6f0/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);
             }