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 2018/03/23 20:34:09 UTC

[4/8] phoenix git commit: PHOENIX-2566 Support NOT NULL constraint for any column for immutable table

PHOENIX-2566 Support NOT NULL constraint for any column for immutable table


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 7d521ed9e8e4790e556d6b3ce978addb5a2a46d1
Parents: 41ea571
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Feb 13 23:14:58 2018 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Fri Mar 23 13:16:59 2018 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |  2 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |  5 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |  4 +-
 .../compile/TupleProjectionCompiler.java        |  3 +-
 .../apache/phoenix/compile/UpsertCompiler.java  | 38 ++++----
 .../phoenix/exception/SQLExceptionCode.java     |  5 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 37 ++++----
 .../org/apache/phoenix/schema/PColumnImpl.java  |  6 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  1 -
 .../org/apache/phoenix/util/SchemaUtil.java     |  6 ++
 .../phoenix/compile/QueryCompilerTest.java      | 98 +++++++++++++++++++-
 12 files changed, 158 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index ab75d34..e6087b8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -477,7 +477,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 stmt.execute();
                 fail("Should have failed since altering a table by adding a non-nullable column is not allowed.");
             } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_ADD_NOT_NULLABLE_COLUMN.getErrorCode(), e.getErrorCode());
+                assertEquals(SQLExceptionCode.KEY_VALUE_NOT_NULL.getErrorCode(), e.getErrorCode());
             } finally {
                 closeStatement(stmt);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 1abc653..491889d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -36,7 +36,6 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GlobalPermissionOrBuilder;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -416,7 +415,7 @@ public class CreateTableIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(ddl);
             fail(" Non pk column ENTRY_POINT_NAME has a NOT NULL constraint");
         } catch (SQLException sqle) {
-            assertEquals(SQLExceptionCode.INVALID_NOT_NULL_CONSTRAINT.getErrorCode(),
+            assertEquals(SQLExceptionCode.KEY_VALUE_NOT_NULL.getErrorCode(),
                 sqle.getErrorCode());
         }
     }
@@ -432,7 +431,7 @@ public class CreateTableIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(ddl);
             fail(" Non pk column V has a NOT NULL constraint");
         } catch (SQLException sqle) {
-            assertEquals(SQLExceptionCode.INVALID_NOT_NULL_CONSTRAINT.getErrorCode(),
+            assertEquals(SQLExceptionCode.KEY_VALUE_NOT_NULL.getErrorCode(),
                 sqle.getErrorCode());
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index 8615bac..7a2d3e1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
 import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.ColumnNotFoundException;
@@ -961,8 +962,7 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
                     "ALTER TABLE " + tableName + " ADD z_string varchar not null primary key");
                 fail();
             } catch (SQLException e) {
-                assertTrue(e.getMessage(), e.getMessage().contains(
-                    "ERROR 1006 (42J04): Only nullable columns may be added to a multi-part row key."));
+                assertEquals(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
             }
             conn1.createStatement().executeUpdate(
                 "ALTER TABLE " + tableName + " ADD z_string varchar primary key");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index d0b900c..fccded2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -183,8 +183,9 @@ public class TupleProjectionCompiler {
                       SchemaUtil.getColumnName(table.getName().getString(), colName) 
                     : SchemaUtil.getColumnName(tableRef.getTableAlias(), colName);
             PName familyName =  SchemaUtil.isPKColumn(sourceColumn) ? (retainPKColumns ? null : PNameFactory.newName(VALUE_COLUMN_FAMILY)) : sourceColumn.getFamilyName();
+            // If we're not retaining the PK columns, then we should switch columns to be nullable
             PColumn column = new ProjectedColumn(PNameFactory.newName(aliasedName), familyName, 
-                    position++, sourceColumn.isNullable(), sourceColumnRef, sourceColumn.getColumnQualifierBytes());
+                    position++, sourceColumn.isNullable() || familyName != null, sourceColumnRef, sourceColumn.getColumnQualifierBytes());
             projectedColumns.add(column);
         }
         EncodedCQCounter cqCounter = EncodedCQCounter.NULL_COUNTER;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/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 d67f858..ae8541c 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
@@ -430,20 +430,20 @@ public class UpsertCompiler {
             targetColumns.addAll(Collections.<PColumn>nCopies(columnIndexesToBe.length, null));
             Arrays.fill(columnIndexesToBe, -1); // TODO: necessary? So we'll get an AIOB exception if it's not replaced
             Arrays.fill(pkSlotIndexesToBe, -1); // TODO: necessary? So we'll get an AIOB exception if it's not replaced
-            BitSet pkColumnsSet = new BitSet(table.getPKColumns().size());
+            BitSet columnsBeingSet = new BitSet(table.getColumns().size());
             int i = 0;
             if (isSharedViewIndex) {
                 PColumn indexIdColumn = table.getPKColumns().get(i + posOffset);
-                columnIndexesToBe[i] = indexIdColumn.getPosition();
-                pkColumnsSet.set(pkSlotIndexesToBe[i] = i + posOffset);
+                columnsBeingSet.set(columnIndexesToBe[i] = indexIdColumn.getPosition());
+                pkSlotIndexesToBe[i] = i + posOffset;
                 targetColumns.set(i, indexIdColumn);
                 i++;
             }
             // Add tenant column directly, as we don't want to resolve it as this will fail
             if (isTenantSpecific) {
                 PColumn tenantColumn = table.getPKColumns().get(i + posOffset);
-                columnIndexesToBe[i] = tenantColumn.getPosition();
-                pkColumnsSet.set(pkSlotIndexesToBe[i] = i + posOffset);
+                columnsBeingSet.set(columnIndexesToBe[i] = tenantColumn.getPosition());
+                pkSlotIndexesToBe[i] = i + posOffset;
                 targetColumns.set(i, tenantColumn);
                 i++;
             }
@@ -458,18 +458,18 @@ public class UpsertCompiler {
                     overlapViewColumnsToBe.add(column);
                     addViewColumnsToBe.remove(column);
                 }
-                columnIndexesToBe[i] = ref.getColumnPosition();
+                columnsBeingSet.set(columnIndexesToBe[i] = ref.getColumnPosition());
                 targetColumns.set(i, column);
                 if (SchemaUtil.isPKColumn(column)) {
-                    pkColumnsSet.set(pkSlotIndexesToBe[i] = ref.getPKSlotPosition());
+                    pkSlotIndexesToBe[i] = ref.getPKSlotPosition();
                 }
                 i++;
             }
             for (PColumn column : addViewColumnsToBe) {
-                columnIndexesToBe[i] = column.getPosition();
+                columnsBeingSet.set(columnIndexesToBe[i] = column.getPosition());
                 targetColumns.set(i, column);
                 if (SchemaUtil.isPKColumn(column)) {
-                    pkColumnsSet.set(pkSlotIndexesToBe[i] = SchemaUtil.getPKPosition(table, column));
+                    pkSlotIndexesToBe[i] = SchemaUtil.getPKPosition(table, column);
                 }
                 i++;
             }
@@ -480,20 +480,18 @@ public class UpsertCompiler {
                 // Need to resize columnIndexesToBe and pkSlotIndexesToBe to include this extra column.
                 columnIndexesToBe = Arrays.copyOf(columnIndexesToBe, columnIndexesToBe.length + 1);
                 pkSlotIndexesToBe = Arrays.copyOf(pkSlotIndexesToBe, pkSlotIndexesToBe.length + 1);
-                columnIndexesToBe[i] = rowTimestampCol.getPosition();
-                pkColumnsSet.set(pkSlotIndexesToBe[i] = table.getRowTimestampColPos());
+                columnsBeingSet.set(columnIndexesToBe[i] = rowTimestampCol.getPosition());
+                pkSlotIndexesToBe[i] = table.getRowTimestampColPos();
                 targetColumns.add(rowTimestampCol);
                 if (valueNodes != null && !valueNodes.isEmpty()) {
                     valueNodes.add(getNodeForRowTimestampColumn(rowTimestampCol));
                 }
                 nColumnsToSet++;
             }
-            for (i = posOffset; i < table.getPKColumns().size(); i++) {
-                PColumn pkCol = table.getPKColumns().get(i);
-                if (!pkColumnsSet.get(i)) {
-                    if (!pkCol.isNullable() && pkCol.getExpressionStr() == null) {
-                        throw new ConstraintViolationException(table.getName().getString() + "." + pkCol.getName().getString() + " may not be null");
-                    }
+            for (i = posOffset; i < table.getColumns().size(); i++) {
+                PColumn column = table.getColumns().get(i);
+                if (!columnsBeingSet.get(i) && !column.isNullable() && column.getExpressionStr() == null) {
+                    throw new ConstraintViolationException(SchemaUtil.getColumnDisplayName(column) + " may not be null");
                 }
             }
         }
@@ -568,6 +566,12 @@ public class UpsertCompiler {
             nColumnsToSet = nValuesToSet;
             columnIndexesToBe = Arrays.copyOf(columnIndexesToBe, nValuesToSet);
             pkSlotIndexesToBe = Arrays.copyOf(pkSlotIndexesToBe, nValuesToSet);
+            for (int i = posOffset + nValuesToSet; i < table.getColumns().size(); i++) {
+                PColumn column = table.getColumns().get(i);
+                if (!column.isNullable() && column.getExpressionStr() == null) {
+                    throw new ConstraintViolationException(SchemaUtil.getColumnDisplayName(column) + " may not be null");
+                }
+            }
         }
         
         if (nValuesToSet != nColumnsToSet) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/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 c798a1c..817e0d1 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
@@ -145,7 +145,6 @@ public enum SQLExceptionCode {
     }),
     ORDER_BY_ARRAY_NOT_SUPPORTED(515, "42893", "ORDER BY of an array type is not allowed."),
     NON_EQUALITY_ARRAY_COMPARISON(516, "42894", "Array types may only be compared using = or !=."),
-    INVALID_NOT_NULL_CONSTRAINT(517, "42895", "Invalid not null constraint on non primary key column."),
 
     /**
      *  Invalid Transaction State (errorcode 05, sqlstate 25)
@@ -203,14 +202,14 @@ public enum SQLExceptionCode {
     PRIMARY_KEY_WITH_FAMILY_NAME(1003, "42J01", "Primary key columns must not have a family name."),
     PRIMARY_KEY_OUT_OF_ORDER(1004, "42J02", "Order of columns in primary key constraint must match the order in which they're declared."),
     VARBINARY_IN_ROW_KEY(1005, "42J03", "The VARBINARY/ARRAY type can only be used as the last part of a multi-part row key."),
-    NOT_NULLABLE_COLUMN_IN_ROW_KEY(1006, "42J04", "Only nullable columns may be added to a multi-part row key."),
+    NOT_NULLABLE_COLUMN_IN_ROW_KEY(1006, "42J04", "Only nullable columns may be added to primary key."),
     VARBINARY_LAST_PK(1015, "42J04", "Cannot add column to table when the last PK column is of type VARBINARY or ARRAY."),
     NULLABLE_FIXED_WIDTH_LAST_PK(1023, "42J04", "Cannot add column to table when the last PK column is nullable and fixed width."),
     CANNOT_MODIFY_VIEW_PK(1036, "42J04", "Cannot modify the primary key of a VIEW if last PK column of parent is variable length."),
     BASE_TABLE_COLUMN(1037, "42J04", "Cannot modify columns of base table used by tenant-specific tables."),
     UNALLOWED_COLUMN_FAMILY(1090, "42J04", "Column family names should not contain local index column prefix: "+QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX),
     // Key/value column related errors
-    KEY_VALUE_NOT_NULL(1007, "42K01", "A key/value column may not be declared as not null."),
+    KEY_VALUE_NOT_NULL(1007, "42K01", "A non primary key column may only be declared as not null on tables with immutable rows."),
     // View related errors.
     VIEW_WITH_TABLE_CONFIG(1008, "42L01", "A view may not contain table configuration properties."),
     VIEW_WITH_PROPERTIES(1009, "42L02", "Properties may not be defined for a view."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index a55af6d..41ed1b2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -488,7 +488,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     private static final PDatum EXPLAIN_PLAN_DATUM = new PDatum() {
         @Override
         public boolean isNullable() {
-            return false;
+            return true;
         }
         @Override
         public PDataType getDataType() {
@@ -513,7 +513,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     public static final String EXPLAIN_PLAN_BYTES_ESTIMATE_COLUMN_ALIAS = "EST_BYTES_READ";
     private static final PColumnImpl EXPLAIN_PLAN_BYTES_ESTIMATE_COLUMN =
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_BYTES_ESTIMATE),
-                    PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, false, 1,
+                    PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 1,
                     SortOrder.getDefault(), 0, null, false, null, false, false,
                     EXPLAIN_PLAN_BYTES_ESTIMATE);
 
@@ -523,7 +523,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     public static final String EXPLAIN_PLAN_ROWS_COLUMN_ALIAS = "EST_ROWS_READ";
     private static final PColumnImpl EXPLAIN_PLAN_ROWS_ESTIMATE_COLUMN =
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ROWS_ESTIMATE),
-                    PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, false, 2,
+                    PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 2,
                     SortOrder.getDefault(), 0, null, false, null, false, false,
                     EXPLAIN_PLAN_ROWS_ESTIMATE);
 
@@ -533,7 +533,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     public static final String EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN_ALIAS = "EST_INFO_TS";
     private static final PColumnImpl EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN =
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ESTIMATE_INFO_TS),
-                PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, false, 3,
+                PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 3,
                 SortOrder.getDefault(), 0, null, false, null, false, false,
                 EXPLAIN_PLAN_ESTIMATE_INFO_TS);
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/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 45e9898..78d6e52 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
@@ -970,7 +970,8 @@ public class MetaDataClient {
         argUpsert.execute();
     }
 
-    private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily, boolean addingToPK, byte[] columnQualifierBytes) throws SQLException {
+    private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily,
+            boolean addingToPK, byte[] columnQualifierBytes, boolean isImmutableRows) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
             SortOrder sortOrder = def.getSortOrder();
@@ -1002,7 +1003,7 @@ public class MetaDataClient {
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
                     .setColumnName(columnName).setFamilyName(family).build().buildException();
-                } else if (!def.isNull()) {
+                } else if (!def.isNull() && !isImmutableRows) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
                     .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
@@ -2216,7 +2217,6 @@ public class MetaDataClient {
             }
 
             Map<String, PName> familyNames = Maps.newLinkedHashMap();
-            boolean isPK = false;
             boolean rowTimeStampColumnAlreadyFound = false;
             int positionOffset = columns.size();
             if (saltBucketNum != null) {
@@ -2307,19 +2307,20 @@ public class MetaDataClient {
             }
 
             Map<String, Integer> changedCqCounters = new HashMap<>(colDefs.size());
+            boolean wasPKDefined = false;
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
-                    if (isPK) {
+                    if (wasPKDefined) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
                             .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                     }
-                    isPK = true;
+                    wasPKDefined = true;
                 } else {
                     // do not allow setting NOT-NULL constraint on non-primary columns.
-                    if (  Boolean.FALSE.equals(colDef.isNull()) &&
-                        ( isPK || ( pkConstraint != null && !pkConstraint.contains(colDef.getColumnDefName())))) {
-                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_NOT_NULL_CONSTRAINT)
+                    if (  !colDef.isNull() && !isImmutableRows &&
+                        ( wasPKDefined || !isPkColumn(pkConstraint, colDef))) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
                                 .setSchemaName(schemaName)
                                 .setTableName(tableName)
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
@@ -2327,7 +2328,7 @@ public class MetaDataClient {
                 }
                 ColumnName columnDefName = colDef.getColumnDefName();
                 String colDefFamily = columnDefName.getFamilyName();
-                boolean isPkColumn = isPkColumn(pkConstraint, colDef, columnDefName);
+                boolean isPkColumn = isPkColumn(pkConstraint, colDef);
                 String cqCounterFamily = null;
                 if (!isPkColumn) {
                     if (immutableStorageScheme == SINGLE_CELL_ARRAY_WITH_OFFSETS && encodingScheme != NON_ENCODED_QUALIFIERS) {
@@ -2348,7 +2349,7 @@ public class MetaDataClient {
                     .setSchemaName(schemaName)
                     .setTableName(tableName).build().buildException();
                 }
-                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false, columnQualifierBytes);
+                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false, columnQualifierBytes, isImmutableRows);
                 if (cqCounter.increment(cqCounterFamily)) {
                     changedCqCounters.put(cqCounterFamily, cqCounter.getNextQualifier(cqCounterFamily));
                 }
@@ -2388,7 +2389,7 @@ public class MetaDataClient {
             }
             
             // We need a PK definition for a TABLE or mapped VIEW
-            if (!isPK && pkColumnsNames.isEmpty() && tableType != PTableType.VIEW && viewType != ViewType.MAPPED) {
+            if (!wasPKDefined && pkColumnsNames.isEmpty() && tableType != PTableType.VIEW && viewType != ViewType.MAPPED) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
                     .setSchemaName(schemaName)
                     .setTableName(tableName)
@@ -2750,8 +2751,8 @@ public class MetaDataClient {
         }
     }
 
-    private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef, ColumnName columnDefName) {
-        return colDef.isPK() || (pkConstraint != null && pkConstraint.getColumnWithSortOrder(columnDefName) != null);
+    private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef) {
+        return colDef.isPK() || (pkConstraint != null && pkConstraint.contains(colDef.getColumnDefName()));
     }
     
     /**
@@ -3243,6 +3244,8 @@ public class MetaDataClient {
 
                 MetaPropertiesEvaluated metaPropertiesEvaluated = new MetaPropertiesEvaluated();
                 changingPhoenixTableProperty = evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
+                // If changing isImmutableRows to true or it's not being changed and is already true
+                boolean willBeImmutableRows = Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || (metaPropertiesEvaluated.getIsImmutableRows() == null && table.isImmutableRows());
 
                 Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || metaProperties.getNonTxToTx());
                 int numPkColumnsAdded = 0;
@@ -3265,8 +3268,8 @@ public class MetaDataClient {
                                 if(colDef.isPK()) {
                                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY)
                                     .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
-                                } else {
-                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ADD_NOT_NULLABLE_COLUMN)
+                                } else if (!willBeImmutableRows) {
+                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
                                     .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                                 }
                             }
@@ -3310,7 +3313,7 @@ public class MetaDataClient {
                                 .setSchemaName(schemaName)
                                 .setTableName(tableName).build().buildException();
                             }
-                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true, columnQualifierBytes);
+                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true, columnQualifierBytes, willBeImmutableRows);
                             columns.add(column);
                             String pkName = null;
                             Short keySeq = null;
@@ -3348,7 +3351,7 @@ public class MetaDataClient {
                                         ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, colDef.getColumnDefName().getColumnName()));
                                         Expression expression = new RowKeyColumnExpression(columns.get(i), new RowKeyValueAccessor(pkColumns, ++pkSlotPosition));
                                         ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), colDef.isNull(), colDef.getMaxLength(), colDef.getScale(), true, colDef.getSortOrder(), expression.toString(), colDef.isRowTimestamp());
-                                        PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true, null);
+                                        PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true, null, willBeImmutableRows);
                                         addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString(), index.getPKName() == null ? null : index.getPKName().getString(), ++nextIndexKeySeq, index.getBucketNum() != null);
                                     }
                                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index 78baa4c..45aca98 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -138,11 +138,7 @@ public class PColumnImpl implements PColumn {
 
     @Override
     public boolean isNullable() {
-        // Only PK columns can be NOT NULL. We prevent this in the
-        // CREATE TABLE statement now (PHOENIX-1266), but this extra
-        // check for familyName != null will ensure that for existing
-        // tables we never treat key value columns as NOT NULL.
-        return nullable || familyName != null;
+        return nullable;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/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 fb30bc7..a7b31e8 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
@@ -69,7 +69,6 @@ import org.apache.phoenix.schema.types.PDataType;
 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.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 51f6ff9..a73467a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -312,6 +312,12 @@ public class SchemaUtil {
         return getName(cf == null || cf.isEmpty() ? null : cf, cq, false);
     }
     
+    public static String getColumnDisplayName(PColumn column) {
+        PName columnName = column.getFamilyName();
+        String cf = columnName == null ? null : columnName.getString();
+        return getName(cf == null || cf.isEmpty() ? null : cf, column.getName().getString(), false);
+    }
+    
     public static String getCaseSensitiveColumnDisplayName(String cf, String cq) {
         return getName(cf == null || cf.isEmpty() ? null : cf, cq, true);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7d521ed9/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 73cd69c..1e615c1 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
@@ -236,13 +236,30 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             statement.execute();
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.INVALID_NOT_NULL_CONSTRAINT.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.KEY_VALUE_NOT_NULL.getErrorCode(), e.getErrorCode());
         } finally {
             conn.close();
         }
     }
 
     @Test
+    public void testImmutableRowsPK() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            String query = "CREATE IMMUTABLE TABLE foo (pk integer not null, col1 decimal, col2 decimal)";
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.execute();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.PRIMARY_KEY_MISSING.getErrorCode(), e.getErrorCode());
+        }
+        String query = "CREATE IMMUTABLE TABLE foo (k1 integer not null, k2 decimal not null, col1 decimal not null, constraint pk primary key (k1,k2))";
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.execute();
+        conn.close();
+    }
+
+    @Test
     public void testUnknownFamilyNameInTableOption() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -1036,6 +1053,25 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testAlterNotNull() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute("ALTER TABLE atable ADD xyz VARCHAR NOT NULL");
+            fail();
+        } catch (SQLException e) { // expected
+            assertEquals(SQLExceptionCode.KEY_VALUE_NOT_NULL.getErrorCode(), e.getErrorCode());
+        }
+        conn.createStatement().execute("CREATE IMMUTABLE TABLE foo (K1 VARCHAR PRIMARY KEY)");
+        try {
+            conn.createStatement().execute("ALTER TABLE foo ADD xyz VARCHAR NOT NULL PRIMARY KEY");
+            fail();
+        } catch (SQLException e) { // expected
+            assertEquals(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
+        }
+        conn.createStatement().execute("ALTER TABLE FOO ADD xyz VARCHAR NOT NULL");
+    }
+
+    @Test
     public void testSubstrSetScanKey() throws Exception {
         String query = "SELECT inst FROM ptsdb WHERE substr(inst, 0, 3) = 'abc'";
         List<Object> binds = Collections.emptyList();
@@ -2711,6 +2747,66 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testNotNullKeyValueColumnSalted() throws Exception {
+        testNotNullKeyValueColumn(3);
+    }
+    @Test
+    public void testNotNullKeyValueColumnUnsalted() throws Exception {
+        testNotNullKeyValueColumn(0);
+    }
+    
+    private void testNotNullKeyValueColumn(int saltBuckets) throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute("CREATE TABLE t1 (k integer not null primary key, v bigint not null) IMMUTABLE_ROWS=true" + (saltBuckets == 0 ? "" : (",SALT_BUCKETS="+saltBuckets)));
+            conn.createStatement().execute("UPSERT INTO t1 VALUES(0)");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+        try {
+            conn.createStatement().execute("CREATE TABLE t2 (k integer not null primary key, v1 bigint not null, v2 varchar, v3 tinyint not null) IMMUTABLE_ROWS=true" + (saltBuckets == 0 ? "" : (",SALT_BUCKETS="+saltBuckets)));
+            conn.createStatement().execute("UPSERT INTO t2(k, v3) VALUES(0,0)");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+        try {
+            conn.createStatement().execute("CREATE TABLE t3 (k integer not null primary key, v1 bigint not null, v2 varchar, v3 tinyint not null) IMMUTABLE_ROWS=true" + (saltBuckets == 0 ? "" : (",SALT_BUCKETS="+saltBuckets)));
+            conn.createStatement().execute("UPSERT INTO t3(k, v1) VALUES(0,0)");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+        conn.createStatement().execute("CREATE TABLE t4 (k integer not null primary key, v1 bigint not null) IMMUTABLE_ROWS=true" + (saltBuckets == 0 ? "" : (",SALT_BUCKETS="+saltBuckets)));
+        conn.createStatement().execute("UPSERT INTO t4 VALUES(0,0)");
+        conn.createStatement().execute("CREATE TABLE t5 (k integer not null primary key, v1 bigint not null default 0) IMMUTABLE_ROWS=true" + (saltBuckets == 0 ? "" : (",SALT_BUCKETS="+saltBuckets)));
+        conn.createStatement().execute("UPSERT INTO t5 VALUES(0)");
+        conn.close();
+    }
+
+    @Test
+    public void testAlterAddNotNullKeyValueColumn() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t1 (k integer not null primary key, v1 bigint not null) IMMUTABLE_ROWS=true");
+        try {
+            conn.createStatement().execute("ALTER TABLE t1 ADD k2 bigint not null primary key");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
+        }
+        conn.createStatement().execute("ALTER TABLE t1 ADD v2 bigint not null");
+        try {
+            conn.createStatement().execute("UPSERT INTO t1(k, v1) VALUES(0,0)");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CONSTRAINT_VIOLATION.getErrorCode(), e.getErrorCode());
+        }
+        conn.createStatement().execute("UPSERT INTO t1 VALUES(0,0,0)");
+        conn.createStatement().execute("UPSERT INTO t1(v1,k,v2) VALUES(0,0,0)");
+    }
+    
+    @Test
     public void testOnDupKeyForImmutableTable() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         try {