You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by jaanai0 <gi...@git.apache.org> on 2018/09/21 12:20:27 UTC

[GitHub] phoenix pull request #355: support alter modify column grammar

GitHub user jaanai0 opened a pull request:

    https://github.com/apache/phoenix/pull/355

    support alter modify column grammar

    #PHOENIX-4815 
    
    @joshelser @twdsilva  Please help me reviewing this patch.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jaanai0/phoenix master

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/phoenix/pull/355.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #355
    
----
commit 7acc7f1512dd2b2df729530dbe9692d8810e0253
Author: Jaanai <cl...@...>
Date:   2018-09-21T12:17:13Z

    support alter modify column grammar

----


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220423727
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4423,113 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    try {
    +                        for (Mutation m : metaData) {
    +
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        // Since the row key is not nullable fixed length, the data type has been translated to variable
    +                        // length when index tables were created, So we will not process data type of index tables.
    --- End diff --
    
    If the table has indexes you need to modify the covered columns that have changed, right? In metadata client you will need to prevent modifying any indexed columns as well. 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220898281
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1277,162 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumn() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
    +            conn.createStatement().execute(
    --- End diff --
    
    Sorry, I didn't forget to test these case.  I saw that a row of query results will be compressed a KeyValue which is byte representation of the `KeyValueSchema`,so this patch will not work,  I will initializing a PR again after fixing the problems.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220900816
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TABLE_TYPE).build().buildException();
    +        }
    +
    +        PColumn oldColumn = null;
    +        for (PColumn column : table.getColumns()) {
    +            if (column.getFamilyName() == null) {
    +                if (column.getName().getString().equals(columnName.getColumnName())) {
    +                    oldColumn = column;
    +                }
    +            } else {
    +                if (column.getName().getString().equals(columnName.getColumnName()) &&
    +                        ((columnName.getFamilyName() != null && column.getFamilyName().getString().equals(columnName.getFamilyName())) ||
    +                                (columnName.getFamilyName() == null && column.getFamilyName().getString().equals(QueryConstants.DEFAULT_COLUMN_FAMILY)))) {
    +                    oldColumn = column;
    +                    break;
    +                }
    +            }
    +        }
    +
    +        if (oldColumn == null) {
    +            throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(),
    +                    columnName.getFamilyName(), columnName.getColumnName());
    +        }
    +
    +        // Comparision of row keys were affected when we changed max length of pk columns to pad more placeholder,
    +        // so we can not modify length of the PK column.
    +        if (oldColumn.isRowTimestamp() || SchemaUtil.isPKColumn(oldColumn)) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN).build().buildException();
    +        }
    +
    --- End diff --
    
    OK


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226303373
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java ---
    @@ -1435,6 +1384,32 @@ public MutationState execute() throws SQLException {
             }
         }
     
    +    private static class ExecutableModifyColumnStatement extends ModifyColumnStatement implements CompilableStatement {
    +
    +        ExecutableModifyColumnStatement(NamedTableNode table, PTableType tableType, ColumnDef columnDef, boolean ifNotExists) {
    +            super(table, tableType, columnDef, ifNotExists);
    +        }
    +
    +        @SuppressWarnings("unchecked")
    +        @Override
    +        public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
    +            final StatementContext context = new StatementContext(stmt);
    +            return new BaseMutationPlan(context, this.getOperation()) {
    +
    +                @Override
    --- End diff --
    
    Ok, I will add a test for the explain plan


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224306623
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java ---
    @@ -384,6 +367,12 @@ public SQLException newException(SQLExceptionInfo info) {
         INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES(1137, "XCL37", "If IMMUTABLE_STORAGE_SCHEME property is not set to ONE_CELL_PER_COLUMN COLUMN_ENCODED_BYTES cannot be 0"),
         INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE(1138, "XCL38", "IMMUTABLE_STORAGE_SCHEME property cannot be changed from/to ONE_CELL_PER_COLUMN "),
         CANNOT_SET_GUIDE_POST_WIDTH(1139, "XCL39", "Guide post width can only be set on base data tables"),
    +
    +    INVALID_DATA_TYPE_CAST(1141, "XCL41", "Invalid data type cast "),
    --- End diff --
    
    Ok, Thanks for your review.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224635037
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1298,7 +1284,322 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testQueryAfterModifiedColumn() throws Exception {
    +        Properties props = new Properties();
    +        String tableName = generateUniqueName();
    +        String indexName = "IDX_" + tableName;
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE TABLE " + tableName
    +                    + " (a VARCHAR(5), b VARCHAR(5), CONSTRAINT PK PRIMARY KEY (a))");
    +            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (b)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('a','12345')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('b','13555')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('c','13666')");
    +            conn.commit();
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify b VARCHAR(2)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('d','13')");
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13'");
    +                assertTrue(rs.next());
    +                assertEquals("13", rs.getString(1));
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13555'");
    +                assertTrue(rs.next());
    +                assertEquals("13555", rs.getString(1));
    --- End diff --
    
    Since we don't modify existings data, when we decrease the length of a VARCHAR column its possible that the string returned is of length larger than expected. I don't think we should allow this to happen. We should support *only* increasing the length and not support decreasing the length of a column.


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
    @twdsilva  I have updated the new patch. Please review, thank you very much! :)


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
    Thanks for your point out.  @dbwong This grammar references ORACLE dialect( https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_3001.htm#i2103956), only one column can be modified at a time.
    
    The modify grammar will not change old data, which just change schema information. we can get correctly query result on the server side as long as the values of filtering conditions are tampered with on the client side.
    
    For Varchar type,  now we can not trim query results if the length of fields is decreased, so the example that @dbwong gave will be ok, for blew queries that haven't result, I also add a new test case in the code which similar with this scenario.
    
    > SELECT * from table WHERE b=='13'; -- does this return 2 rows? or does it return 0?
    SELECT * from tableIdx WHERE b=='13'; -- this will return 0 rows
    
    For Char type, we will forbid to decrease the length of fields. The query results will be trimmed on the client side that can not match actual data. For example:
    
    _CREATE TABLE  tableName  (a VARCHAR(5), b VARCHAR(5), CONSTRAINT PK PRIMARY KEY (a));
    CREATE INDEX tableIdx ON  tableName  (b);
    UPSERT INTO  tableName Values('b','13555');
    ALTER TABLE  tableName  modify b VARCHAR(2);
    UPSERT INTO tableName Values('d','13');
    SELECT /*+ NO_INDEX*/ b from  tableName  WHERE b='13';
    SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13555';
    SELECT \"0:B\" from tableIdx WHERE \"0:B\"='13555';
    SELECT \"0:B\" from tableIdx WHERE \"0:B\"='13';_
    
    Output:
    13
    13(this is a incorrect result)
    13555
    13
    
    @twdsilva @dbwong  I have updated the patch again.
    



---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220424066
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TABLE_TYPE).build().buildException();
    +        }
    +
    +        PColumn oldColumn = null;
    +        for (PColumn column : table.getColumns()) {
    +            if (column.getFamilyName() == null) {
    +                if (column.getName().getString().equals(columnName.getColumnName())) {
    +                    oldColumn = column;
    +                }
    +            } else {
    +                if (column.getName().getString().equals(columnName.getColumnName()) &&
    +                        ((columnName.getFamilyName() != null && column.getFamilyName().getString().equals(columnName.getFamilyName())) ||
    +                                (columnName.getFamilyName() == null && column.getFamilyName().getString().equals(QueryConstants.DEFAULT_COLUMN_FAMILY)))) {
    +                    oldColumn = column;
    +                    break;
    +                }
    +            }
    +        }
    +
    +        if (oldColumn == null) {
    +            throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(),
    +                    columnName.getFamilyName(), columnName.getColumnName());
    +        }
    +
    +        // Comparision of row keys were affected when we changed max length of pk columns to pad more placeholder,
    +        // so we can not modify length of the PK column.
    +        if (oldColumn.isRowTimestamp() || SchemaUtil.isPKColumn(oldColumn)) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN).build().buildException();
    +        }
    +
    --- End diff --
    
    Also add a check if the column is from a view that it is not derived (i.e. inherited from the parent table). You should thrown an exception in this case.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r221131188
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2853,12 +2687,13 @@ MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
                         return result;
                     } else {
                         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, false, false, null);
    -                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
    --- End diff --
    
    You can add a new COLUMN_MODIFIED MutationCode, but no need to change the existing MutationCode returned for drop column etc.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224210097
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java ---
    @@ -384,6 +367,12 @@ public SQLException newException(SQLExceptionInfo info) {
         INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES(1137, "XCL37", "If IMMUTABLE_STORAGE_SCHEME property is not set to ONE_CELL_PER_COLUMN COLUMN_ENCODED_BYTES cannot be 0"),
         INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE(1138, "XCL38", "IMMUTABLE_STORAGE_SCHEME property cannot be changed from/to ONE_CELL_PER_COLUMN "),
         CANNOT_SET_GUIDE_POST_WIDTH(1139, "XCL39", "Guide post width can only be set on base data tables"),
    +
    +    INVALID_DATA_TYPE_CAST(1141, "XCL41", "Invalid data type cast "),
    --- End diff --
    
    Can you change the message to "cannot modify column because of invalid data type cast"


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224182097
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1276,272 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumnWithIndexTables() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +
    +            conn.createStatement().execute("DROP TABLE IF EXISTS " + tableName);
    --- End diff --
    
    The drop table isn't required since we always create tables with unique names.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220899258
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4423,113 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    try {
    +                        for (Mutation m : metaData) {
    +
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        // Since the row key is not nullable fixed length, the data type has been translated to variable
    +                        // length when index tables were created, So we will not process data type of index tables.
    --- End diff --
    
    Yes


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220894529
  
    --- Diff: phoenix-core/src/main/antlr3/PhoenixSQL.g ---
    @@ -655,9 +656,17 @@ alter_session_node returns [AlterSessionStatement ret]
     
     // Parse an alter table statement.
     alter_table_node returns [AlterTableStatement ret]
    -    :   ALTER (TABLE | v=VIEW) t=from_table_name
    -        ( (DROP COLUMN (IF ex=EXISTS)? c=column_names) | (ADD (IF NOT ex=EXISTS)? (d=column_defs) (p=fam_properties)?) | (SET (p=fam_properties)) )
    -        { PTableType tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW; ret = ( c == null ? factory.addColumn(factory.namedTable(null,t), tt, d, ex!=null, p) : factory.dropColumn(factory.namedTable(null,t), tt, c, ex!=null) ); }
    +@init { PTableType tt = null;}
    +    :   ALTER (TABLE | v=VIEW) t=from_table_name { tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW;}
    +            (
    +                DROP COLUMN (IF ex=EXISTS)? c=column_names { ret = factory.dropColumn(factory.namedTable(null, t), tt, c, ex!=null);}
    +                |
    +                    (
    +                        ADD (IF NOT ex=EXISTS)? cds=column_defs (p=fam_properties)?
    +                        | SET p=fam_properties
    +                     ) { ret = factory.addColumn(factory.namedTable(null, t), tt, cds, ex!=null, p);}
    +                | MODIFY (IF NOT ex=EXISTS)? cd=column_def { ret = factory.modifyColumn(factory.namedTable(null, t), tt, cd, ex!=null);}
    --- End diff --
    
    Yep, that will be confirmed in MetaDataClient
    ```java
     if (oldColumn == null) {
                throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(),
                        columnName.getFamilyName(), columnName.getColumnName());
            }
    ```


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by dbwong <gi...@git.apache.org>.
Github user dbwong commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226150172
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java ---
    @@ -1435,6 +1384,32 @@ public MutationState execute() throws SQLException {
             }
         }
     
    +    private static class ExecutableModifyColumnStatement extends ModifyColumnStatement implements CompilableStatement {
    +
    +        ExecutableModifyColumnStatement(NamedTableNode table, PTableType tableType, ColumnDef columnDef, boolean ifNotExists) {
    +            super(table, tableType, columnDef, ifNotExists);
    +        }
    +
    +        @SuppressWarnings("unchecked")
    +        @Override
    +        public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
    +            final StatementContext context = new StatementContext(stmt);
    +            return new BaseMutationPlan(context, this.getOperation()) {
    +
    +                @Override
    --- End diff --
    
    Was there a test for the explain plan?


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220898860
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/DataTypeCastException.java ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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 org.apache.phoenix.exception.SQLExceptionCode;
    +import org.apache.phoenix.exception.SQLExceptionInfo;
    +
    +import java.sql.SQLException;
    +
    +public class DataTypeCastException extends SQLException {
    +    private static final long serialVersionUID = 1L;
    +    private static SQLExceptionCode code = SQLExceptionCode.INVALID_DATA_TYPE_CAST;
    +
    +    public DataTypeCastException(String newColumn, String oldColumn) {
    --- End diff --
    
    OK 👍 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220420648
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/DataTypeCastException.java ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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 org.apache.phoenix.exception.SQLExceptionCode;
    +import org.apache.phoenix.exception.SQLExceptionInfo;
    +
    +import java.sql.SQLException;
    +
    +public class DataTypeCastException extends SQLException {
    +    private static final long serialVersionUID = 1L;
    +    private static SQLExceptionCode code = SQLExceptionCode.INVALID_DATA_TYPE_CAST;
    +
    +    public DataTypeCastException(String newColumn, String oldColumn) {
    --- End diff --
    
    Maybe also include the incompatible data types here


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224466779
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4424,195 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    Cell dataTypeCell = null;
    +                    Cell columnSizeCell = null;
    +                    Cell decimalDigitCell = null;
    +                    List<byte[]> mutatedTableNames = new ArrayList<>();
    +                    byte[] familyName = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
    +
    +                    try {
    +                        for (Mutation m : metaData) {
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            List<Cell> cells;
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                                cells = m.getFamilyCellMap().get(column.getFamilyName().getBytes());
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                                cells = m.getFamilyCellMap().get(familyName);
    +                            }
    +
    +                            for (Cell cell : cells) {
    +                                if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DATA_TYPE_BYTES) == 0) {
    +                                    dataTypeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES) == 0) {
    +                                    columnSizeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES) == 0) {
    +                                    decimalDigitCell = cell;
    +                                }
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        if (!table.getIndexes().isEmpty()) {
    +                            PhoenixConnection connection = null;
    +                            try {
    +                                connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                            } catch (ClassNotFoundException e) {
    +                            }
    +
    +                            for (PTable index : table.getIndexes()) {
    +                                byte[] tenantIdBytes = index.getTenantId() == null ?
    +                                                ByteUtil.EMPTY_BYTE_ARRAY :
    +                                                index.getTenantId().getBytes();
    +                                byte[] schemaNameBytes = index.getSchemaName().getBytes();
    +                                byte[] indexName = index.getTableName().getBytes();
    +                                byte[] indexKey = SchemaUtil.getTableKey(tenantIdBytes, schemaNameBytes, indexName);
    +
    +                                IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
    +                                boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(
    +                                                new Pair<>(column.getFamilyName().getString(), column.getName().getString()));
    +                                ColumnReference coveredColumn = indexMaintainer.getCoveredColumnsOfIndexTable(
    +                                                new ColumnReference(column.getFamilyName().getBytes(), column.getColumnQualifierBytes()));
    +
    +                                // Since the columns of fixed length which in present in primary key of index table will be converted
    +                                // to variable length when index tables are created, So we will not process indexed columns.
    +                                if (isColumnIndexed) {
    +                                    // do nothing
    +                                }
    +
    +                                // Modify length/scala of covered columns
    --- End diff --
    
    ok


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224639651
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java ---
    @@ -121,7 +122,15 @@ public boolean isNull(int position, ValueBitSet bitSet) {
                             int nBytes = ptr.getLength();
                             b = ensureSize(b, offset, offset + nBytes);
                             System.arraycopy(ptr.get(), ptr.getOffset(), b, offset, nBytes);
    -                        offset += nBytes;
    +                        if (field.getMaxLength() != null && field.getMaxLength() > 0 ) {
    --- End diff --
    
    Is this change required to handle existing data of type char after we increase the length? We should only do this padding if the type is PChar right? Is it possible to change the length of any other fixed length data type?


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r221420128
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TABLE_TYPE).build().buildException();
    +        }
    +
    +        PColumn oldColumn = null;
    +        for (PColumn column : table.getColumns()) {
    +            if (column.getFamilyName() == null) {
    +                if (column.getName().getString().equals(columnName.getColumnName())) {
    +                    oldColumn = column;
    +                }
    +            } else {
    +                if (column.getName().getString().equals(columnName.getColumnName()) &&
    +                        ((columnName.getFamilyName() != null && column.getFamilyName().getString().equals(columnName.getFamilyName())) ||
    +                                (columnName.getFamilyName() == null && column.getFamilyName().getString().equals(QueryConstants.DEFAULT_COLUMN_FAMILY)))) {
    +                    oldColumn = column;
    +                    break;
    +                }
    +            }
    +        }
    +
    +        if (oldColumn == null) {
    +            throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(),
    +                    columnName.getFamilyName(), columnName.getColumnName());
    +        }
    +
    +        // Comparision of row keys were affected when we changed max length of pk columns to pad more placeholder,
    +        // so we can not modify length of the PK column.
    +        if (oldColumn.isRowTimestamp() || SchemaUtil.isPKColumn(oldColumn)) {
    --- End diff --
    
    There is no effect.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224467327
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1276,272 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumnWithIndexTables() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +
    +            conn.createStatement().execute("DROP TABLE IF EXISTS " + tableName);
    +            conn.createStatement().execute("CREATE TABLE " + tableName +
    +                    " (ID CHAR(3) NOT NULL, COL1 VARCHAR(5), COL2 VARCHAR(3), COL3 CHAR(4), COL4 CHAR(4)"
    +                    + " CONSTRAINT PKVIEW PRIMARY KEY(ID, COL1)) " + tableDDLOptions);
    +            conn.createStatement().execute("CREATE INDEX IDX_COL2 ON "+ tableName + "(COL2) INCLUDE(COL3, COL4)");
    +
    +            conn.createStatement().executeUpdate("UPSERT INTO " + tableName + " VALUES('123','12345','123','1234','1444')");
    +
    +            //Increasing char length
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify COL3 CHAR(6)");
    --- End diff --
    
    I got it


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224207564
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1276,272 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumnWithIndexTables() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +
    +            conn.createStatement().execute("DROP TABLE IF EXISTS " + tableName);
    +            conn.createStatement().execute("CREATE TABLE " + tableName +
    +                    " (ID CHAR(3) NOT NULL, COL1 VARCHAR(5), COL2 VARCHAR(3), COL3 CHAR(4), COL4 CHAR(4)"
    +                    + " CONSTRAINT PKVIEW PRIMARY KEY(ID, COL1)) " + tableDDLOptions);
    +            conn.createStatement().execute("CREATE INDEX IDX_COL2 ON "+ tableName + "(COL2) INCLUDE(COL3, COL4)");
    +
    +            conn.createStatement().executeUpdate("UPSERT INTO " + tableName + " VALUES('123','12345','123','1234','1444')");
    +
    +            //Increasing char length
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify COL3 CHAR(6)");
    --- End diff --
    
    Can you also test for increasing the length of an indexed column? Also can you please add a comment saying that this is supported because we convert fixed length columns to variable length while creating the index.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226302930
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java ---
    @@ -327,6 +328,9 @@ public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
                 rhsLength = StringUtil.getUnpaddedCharLength(rhsBytes, rhsOffset, rhsLength, rhsSortOrder);
             }
             if (lhsDataType == PChar.INSTANCE) {
    +            // Due to length of PChar columns may be modified, in order to the values of filters can match original data,
    +            // we make rhsLength equal with lhsLength.
    --- End diff --
    
    Thanks for your point out. this approach will not impact Like, InList and RVC etc, because this is implemented in ComparisonExpression, but as you said that I should consider these expressions how correctly evaluate.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220423833
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2853,12 +2687,13 @@ MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
                         return result;
                     } else {
                         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, false, false, null);
    -                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
    --- End diff --
    
    Is there a reason the MutationCode returned needs to change?


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224636852
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java ---
    @@ -17,29 +17,10 @@
      */
     package org.apache.phoenix.jdbc;
     
    -import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_SQL_COUNTER;
    --- End diff --
    
    Don't reorder imports.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220413373
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1277,162 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumn() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
    +            conn.createStatement().execute(
    --- End diff --
    
    What happens if you decrease the length of a varchar column or decreasing the scale and/or precision of a decimal column that has existing data? 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224206646
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4424,195 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    Cell dataTypeCell = null;
    +                    Cell columnSizeCell = null;
    +                    Cell decimalDigitCell = null;
    +                    List<byte[]> mutatedTableNames = new ArrayList<>();
    +                    byte[] familyName = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
    +
    +                    try {
    +                        for (Mutation m : metaData) {
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            List<Cell> cells;
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                                cells = m.getFamilyCellMap().get(column.getFamilyName().getBytes());
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                                cells = m.getFamilyCellMap().get(familyName);
    +                            }
    +
    +                            for (Cell cell : cells) {
    +                                if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DATA_TYPE_BYTES) == 0) {
    +                                    dataTypeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES) == 0) {
    +                                    columnSizeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES) == 0) {
    +                                    decimalDigitCell = cell;
    +                                }
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        if (!table.getIndexes().isEmpty()) {
    +                            PhoenixConnection connection = null;
    +                            try {
    +                                connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                            } catch (ClassNotFoundException e) {
    +                            }
    +
    +                            for (PTable index : table.getIndexes()) {
    +                                byte[] tenantIdBytes = index.getTenantId() == null ?
    +                                                ByteUtil.EMPTY_BYTE_ARRAY :
    +                                                index.getTenantId().getBytes();
    +                                byte[] schemaNameBytes = index.getSchemaName().getBytes();
    +                                byte[] indexName = index.getTableName().getBytes();
    +                                byte[] indexKey = SchemaUtil.getTableKey(tenantIdBytes, schemaNameBytes, indexName);
    +
    +                                IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
    +                                boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(
    +                                                new Pair<>(column.getFamilyName().getString(), column.getName().getString()));
    +                                ColumnReference coveredColumn = indexMaintainer.getCoveredColumnsOfIndexTable(
    +                                                new ColumnReference(column.getFamilyName().getBytes(), column.getColumnQualifierBytes()));
    +
    +                                // Since the columns of fixed length which in present in primary key of index table will be converted
    +                                // to variable length when index tables are created, So we will not process indexed columns.
    +                                if (isColumnIndexed) {
    +                                    // do nothing
    +                                }
    +
    +                                // Modify length/scala of covered columns
    --- End diff --
    
    typo: scale


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224635422
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1298,7 +1284,322 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testQueryAfterModifiedColumn() throws Exception {
    +        Properties props = new Properties();
    +        String tableName = generateUniqueName();
    +        String indexName = "IDX_" + tableName;
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE TABLE " + tableName
    +                    + " (a VARCHAR(5), b VARCHAR(5), CONSTRAINT PK PRIMARY KEY (a))");
    +            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (b)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('a','12345')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('b','13555')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('c','13666')");
    +            conn.commit();
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify b VARCHAR(2)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('d','13')");
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13'");
    +                assertTrue(rs.next());
    +                assertEquals("13", rs.getString(1));
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13555'");
    +                assertTrue(rs.next());
    +                assertEquals("13555", rs.getString(1));
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT \"0:B\" from " + indexName + " WHERE \"0:B\"='13555'");
    +                assertTrue(rs.next());
    +                assertEquals("13555", rs.getString(1));
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT \"0:B\" from " + indexName + " WHERE \"0:B\"='13'");
    +                assertTrue(rs.next());
    +                assertEquals("13", rs.getString(1));
    +            }
    +        }
    +    }
    +
    +    @Test
    +    public void testModifiedCharType() throws Exception {
    +        Properties props = new Properties();
    +        String tableName = generateUniqueName();
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE TABLE " + tableName + " (a char(5), b char(5), CONSTRAINT PK PRIMARY KEY (a))");
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify b char(2)");
    +        }  catch (SQLException e) {
    +            assertEquals(SQLExceptionCode.DISALLOW_DECREASE_CHAR_LENGTH.getErrorCode(), e.getErrorCode());
    +        }
    +    }
    +
    +
    +    @Test
    +    public void testModifyColumnWithIndexTables() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +            conn.createStatement().execute("CREATE TABLE " + tableName +
    +                    " (ID CHAR(3) NOT NULL, COL1 VARCHAR(5), COL2 VARCHAR(3), COL3 CHAR(4), COL4 CHAR(4)"
    +                    + " CONSTRAINT PKVIEW PRIMARY KEY(ID, COL1)) " + tableDDLOptions);
    +            conn.createStatement().execute("CREATE INDEX IDX_COL2 ON "+ tableName + "(COL2) INCLUDE(COL3, COL4)");
    +
    +            conn.createStatement().executeUpdate("UPSERT INTO " + tableName + " VALUES('123','12345','123','1234','1444')");
    +
    +            //Increasing char length
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify COL3 CHAR(6)");
    +            //Increasing char length of the indexed column
    +            //Indexed columns will convert fixed length columns to variable length while creating the index,
    +            //but length/scale of field still had been reserved old values.
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify COL2 CHAR(5)");
    +
    +            conn.createStatement().executeUpdate("UPSERT INTO " + tableName + " VALUES('124','12345','12345','123456','12')");
    +
    +            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
    +            PTable table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), tableName));
    +            assertColumnModify("COL3", table.getSchemaName().getString(),
    +                    table.getTableName().getString(), 6);
    +            assertColumnModify(QueryConstants.DEFAULT_COLUMN_FAMILY + ":" + "COL3", table.getSchemaName().getString(),
    +                    "IDX_COL2", 6);
    +            //char length of the indexed column will not change
    +            assertColumnModify(QueryConstants.DEFAULT_COLUMN_FAMILY + ":" + "COL2", table.getSchemaName().getString(),
    +                    "IDX_COL2", 3);
    +
    +            {
    +                assertEquals(QueryUtil.getExplainPlan(conn.createStatement().executeQuery(" explain select * from " + tableName)),
    +                        "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + schemaName + ":IDX_COL2");
    +
    +                // Getting query results by index table
    +                ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName);
    +                assertTrue(rs.next());
    +                assertEquals("123", rs.getString(1));
    +                assertEquals("12345", rs.getString(2));
    +                // The length of column col2 added 2,
    +                assertEquals("123  ", rs.getString(3));
    +                // The length of column col3 added 2
    +                assertEquals("1234  ", rs.getString(4));
    +                assertEquals("1444", rs.getString(5));
    +                assertTrue(rs.next());
    +                assertEquals("124", rs.getString(1));
    +                assertEquals("12345", rs.getString(2));
    +                assertEquals("12345", rs.getString(3));
    +                assertEquals("123456", rs.getString(4));
    +                assertEquals("12  ", rs.getString(5));
    +                assertFalse(rs.next());
    +                rs.close();
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("select * from " + schemaName + ".IDX_COL2");
    +                assertTrue(rs.next());
    +                // Col2 as a index column that length of type was changed,
    +                // but the result length will not change.
    +                assertEquals("123", rs.getString(1));
    +                assertEquals("123", rs.getString(2));
    +                assertEquals("12345", rs.getString(3));
    +                // Col3 as a covered column which the length of type was increased,
    +                // the result length has increased too.
    +                assertEquals("1234  ", rs.getString(4));
    +                assertEquals("1444", rs.getString(5));
    +                assertTrue(rs.next());
    +                assertEquals("12345", rs.getString(1));
    +                assertEquals("124", rs.getString(2));
    +                assertEquals("12345", rs.getString(3));
    +                assertEquals("123456", rs.getString(4));
    +                assertEquals("12  ", rs.getString(5));
    +                assertFalse(rs.next());
    +                rs.close();
    +            }
    +
    +            conn.close();
    +        }
    --- End diff --
    
    Can you also test a point look-up to verify it works "SELECT ... WHERE COL3='...' "


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224636691
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -17,108 +17,20 @@
      */
     package org.apache.phoenix.coprocessor;
     
    -import static org.apache.hadoop.hbase.KeyValueUtil.createFirstOnRow;
    --- End diff --
    
    Please use the code formatter/templates from https://phoenix.apache.org/develop.html so that the imports are reordered unnecessarily. 


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
    @jaanai0  From the example that @dbwong gave, I don't think we can support decreasing the length of a column. If the column is present in an index this can cause queries to return different results when the index is queried vs the data table. 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226383941
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java ---
    @@ -327,6 +328,9 @@ public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
                 rhsLength = StringUtil.getUnpaddedCharLength(rhsBytes, rhsOffset, rhsLength, rhsSortOrder);
             }
             if (lhsDataType == PChar.INSTANCE) {
    +            // Due to length of PChar columns may be modified, in order to the values of filters can match original data,
    +            // we make rhsLength equal with lhsLength.
    --- End diff --
    
    In Comparison.evaluate() when a value is being read from a column whose length is modified, do you get the original value or the modified value (either a substring or with extra padding)? 
    Please add a test for this.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224673577
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java ---
    @@ -242,19 +242,6 @@ public static Expression create(CompareOp op, List<Expression> children, Immutab
                       }
                     }
                 }
    -
    --- End diff --
    
    That being removed for decresing length of columns, i will revert it.



---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220420145
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    --- End diff --
    
    just check if the type is TABLE or VIEW


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by dbwong <gi...@git.apache.org>.
Github user dbwong commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226149926
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java ---
    @@ -327,6 +328,9 @@ public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
                 rhsLength = StringUtil.getUnpaddedCharLength(rhsBytes, rhsOffset, rhsLength, rhsSortOrder);
             }
             if (lhsDataType == PChar.INSTANCE) {
    +            // Due to length of PChar columns may be modified, in order to the values of filters can match original data,
    +            // we make rhsLength equal with lhsLength.
    --- End diff --
    
    I’m a bit scared of this approach.  There are several comparison like behavior and other row construction in expressions like Like, InList, RVCs, Case that may be impacted as well. 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220898736
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    --- End diff --
    
    what else type of tables should be modified? I think just TABLE and VIEW are be modified


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by dbwong <gi...@git.apache.org>.
Github user dbwong commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
    I'll try to take another look at this sometime tomorrow.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224306474
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java ---
    @@ -381,10 +371,10 @@ public static MetaDataMutationResult constructFromProto(MetaDataResponse proto)
                   result.functions.add(PFunction.createFromProto(function));
               }
               if (proto.getTablesToDeleteCount() > 0) {
    -            result.tableNamesToDelete =
    +            result.mutatedTableNames =
                     Lists.newArrayListWithExpectedSize(proto.getTablesToDeleteCount());
    --- End diff --
    
    Yep,  it will not affect backward compatibility.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224466632
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1294,7 +1276,272 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyColumnWithIndexTables() throws Exception {
    +        String schemaName = generateUniqueName();
    +        String baseTableName =  generateUniqueName();
    +        String tableName = schemaName + "." + baseTableName;
    +        Properties props = new Properties();
    +        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
    +
    +            conn.createStatement().execute("DROP TABLE IF EXISTS " + tableName);
    --- End diff --
    
    ok


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by dbwong <gi...@git.apache.org>.
Github user dbwong commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226148449
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4626,4 +4628,189 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    --- End diff --
    
    Nit: prefer build or get<type> for builder pattern.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r221130667
  
    --- Diff: phoenix-core/src/main/antlr3/PhoenixSQL.g ---
    @@ -655,9 +656,17 @@ alter_session_node returns [AlterSessionStatement ret]
     
     // Parse an alter table statement.
     alter_table_node returns [AlterTableStatement ret]
    -    :   ALTER (TABLE | v=VIEW) t=from_table_name
    -        ( (DROP COLUMN (IF ex=EXISTS)? c=column_names) | (ADD (IF NOT ex=EXISTS)? (d=column_defs) (p=fam_properties)?) | (SET (p=fam_properties)) )
    -        { PTableType tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW; ret = ( c == null ? factory.addColumn(factory.namedTable(null,t), tt, d, ex!=null, p) : factory.dropColumn(factory.namedTable(null,t), tt, c, ex!=null) ); }
    +@init { PTableType tt = null;}
    +    :   ALTER (TABLE | v=VIEW) t=from_table_name { tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW;}
    +            (
    +                DROP COLUMN (IF ex=EXISTS)? c=column_names { ret = factory.dropColumn(factory.namedTable(null, t), tt, c, ex!=null);}
    +                |
    +                    (
    +                        ADD (IF NOT ex=EXISTS)? cds=column_defs (p=fam_properties)?
    +                        | SET p=fam_properties
    +                     ) { ret = factory.addColumn(factory.namedTable(null, t), tt, cds, ex!=null, p);}
    +                | MODIFY (IF NOT ex=EXISTS)? cd=column_def { ret = factory.modifyColumn(factory.namedTable(null, t), tt, cd, ex!=null);}
    --- End diff --
    
    I meant that you don't need to add the "IF NOT EXISTS" to the grammar.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224649318
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java ---
    @@ -1298,7 +1284,322 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
                 assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             }
         }
    -	
    +
    +
    +    @Test
    +    public void testModifyingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a row_timestamp column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testModifyingPKColumnNotAllowedViaAlterTable() throws Exception {
    +        try (Connection conn = DriverManager.getConnection(getUrl())) {
    +            conn.createStatement().execute("CREATE TABLE " + dataTableFullName +
    +                    " (PK1 DATE NOT NULL PRIMARY KEY, PK2 VARCHAR , KV1 VARCHAR) " + tableDDLOptions);
    +            try {
    +                conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " modify PK1 BIGINT");
    +                fail("Altering table to modify a PK column should fail");
    +            } catch (SQLException e) {
    +                assertEquals(SQLExceptionCode.DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN.getErrorCode(), e.getErrorCode());
    +            }
    +            conn.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testQueryAfterModifiedColumn() throws Exception {
    +        Properties props = new Properties();
    +        String tableName = generateUniqueName();
    +        String indexName = "IDX_" + tableName;
    +        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
    +            conn.setAutoCommit(true);
    +            conn.createStatement().execute("CREATE TABLE " + tableName
    +                    + " (a VARCHAR(5), b VARCHAR(5), CONSTRAINT PK PRIMARY KEY (a))");
    +            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (b)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('a','12345')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('b','13555')");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('c','13666')");
    +            conn.commit();
    +            conn.createStatement().execute("ALTER TABLE " + tableName + " modify b VARCHAR(2)");
    +            conn.createStatement().execute("UPSERT INTO " + tableName + " Values('d','13')");
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13'");
    +                assertTrue(rs.next());
    +                assertEquals("13", rs.getString(1));
    +            }
    +
    +            {
    +                ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX*/ b from " + tableName + " WHERE b='13555'");
    +                assertTrue(rs.next());
    +                assertEquals("13555", rs.getString(1));
    --- End diff --
    
    Ok, I agree.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224637219
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java ---
    @@ -381,10 +371,10 @@ public static MetaDataMutationResult constructFromProto(MetaDataResponse proto)
                   result.functions.add(PFunction.createFromProto(function));
               }
               if (proto.getTablesToDeleteCount() > 0) {
    -            result.tableNamesToDelete =
    +            result.mutatedTableNames =
                     Lists.newArrayListWithExpectedSize(proto.getTablesToDeleteCount());
    --- End diff --
    
    Rename tablesToDelete in the MetaDataResponse proto to mutatedTables


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220900300
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2853,12 +2687,13 @@ MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
                         return result;
                     } else {
                         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, false, false, null);
    -                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
    --- End diff --
    
    `TABLE_ALREADY_EXISTS ` as positive MutationCode is wired, so I want to add an `OK` code to identifying success. what do you think about this?


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
     I had updated this patch and added some test. Please review :) @twdsilva 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224208120
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4424,195 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    Cell dataTypeCell = null;
    +                    Cell columnSizeCell = null;
    +                    Cell decimalDigitCell = null;
    +                    List<byte[]> mutatedTableNames = new ArrayList<>();
    +                    byte[] familyName = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
    +
    +                    try {
    +                        for (Mutation m : metaData) {
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            List<Cell> cells;
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                                cells = m.getFamilyCellMap().get(column.getFamilyName().getBytes());
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                                cells = m.getFamilyCellMap().get(familyName);
    +                            }
    +
    +                            for (Cell cell : cells) {
    +                                if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DATA_TYPE_BYTES) == 0) {
    +                                    dataTypeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES) == 0) {
    +                                    columnSizeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES) == 0) {
    +                                    decimalDigitCell = cell;
    +                                }
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        if (!table.getIndexes().isEmpty()) {
    +                            PhoenixConnection connection = null;
    +                            try {
    +                                connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                            } catch (ClassNotFoundException e) {
    +                            }
    +
    +                            for (PTable index : table.getIndexes()) {
    +                                byte[] tenantIdBytes = index.getTenantId() == null ?
    +                                                ByteUtil.EMPTY_BYTE_ARRAY :
    +                                                index.getTenantId().getBytes();
    +                                byte[] schemaNameBytes = index.getSchemaName().getBytes();
    +                                byte[] indexName = index.getTableName().getBytes();
    +                                byte[] indexKey = SchemaUtil.getTableKey(tenantIdBytes, schemaNameBytes, indexName);
    +
    +                                IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
    +                                boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(
    +                                                new Pair<>(column.getFamilyName().getString(), column.getName().getString()));
    +                                ColumnReference coveredColumn = indexMaintainer.getCoveredColumnsOfIndexTable(
    +                                                new ColumnReference(column.getFamilyName().getBytes(), column.getColumnQualifierBytes()));
    +
    +                                // Since the columns of fixed length which in present in primary key of index table will be converted
    +                                // to variable length when index tables are created, So we will not process indexed columns.
    +                                if (isColumnIndexed) {
    --- End diff --
    
    remove this boolean as its not used, just keep the comment about why we don't need to do anything for indexed columns.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224666108
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java ---
    @@ -121,7 +122,15 @@ public boolean isNull(int position, ValueBitSet bitSet) {
                             int nBytes = ptr.getLength();
                             b = ensureSize(b, offset, offset + nBytes);
                             System.arraycopy(ptr.get(), ptr.getOffset(), b, offset, nBytes);
    -                        offset += nBytes;
    +                        if (field.getMaxLength() != null && field.getMaxLength() > 0 ) {
    --- End diff --
    
    @twdsilva  Yes, we should only pad Pchar type, through other data types of fixed width which excepts for PChar have a permanent length of columns. But in this cases that cast type of fixed columns, the results may be incorrect.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r221130895
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    --- End diff --
    
    I misread this, ignore my previous comment.


---

[GitHub] phoenix issue #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on the issue:

    https://github.com/apache/phoenix/pull/355
  
    I had forbidden to decrease the length of columns. @twdsilva @dbwong  @ChinmaySKulkarni 


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r226379287
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4626,4 +4628,189 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    --- End diff --
    
    I agree, rename to build()


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by jaanai0 <gi...@git.apache.org>.
Github user jaanai0 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224306505
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -4588,4 +4424,195 @@ private TableName getParentPhysicalTableName(PTable table) {
                                                                     table.getTableName(), table.isNamespaceMapped())
                                                             .getBytes());
         }
    +
    +    private class TableBuilder {
    +        private Region region;
    +        private byte[] tableKey;
    +        private Integer clientVersion;
    +
    +        public TableBuilder setRegion(Region region) {
    +            this.region = region;
    +            return this;
    +        }
    +
    +        public TableBuilder setTableKey(byte[] tableKey) {
    +            this.tableKey = tableKey;
    +            return this;
    +        }
    +
    +        public TableBuilder setClientVersion(Integer clientVersion) {
    +            this.clientVersion = clientVersion;
    +            return this;
    +        }
    +
    +        public PTable run() throws Exception {
    +            Preconditions.checkNotNull(region);
    +            Preconditions.checkNotNull(tableKey);
    +            Preconditions.checkNotNull(clientVersion);
    +            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
    +            return buildTable(tableKey, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
    +                    false, false, null);
    +        }
    +    }
    +
    +
    +    /**
    +     * Indexes table and View will not be modify when the parent tables are modified, so modify has a simple logic in server side.
    +     * @param controller
    +     * @param request
    +     * @param done
    +     */
    +    @Override
    +    public void modifyColumn(RpcController controller, final MetaDataProtos.ModifyColumnRequest request,
    +            RpcCallback<MetaDataResponse> done) {
    +        try {
    +            final List<Mutation> metaData = ProtobufUtil.getMutations(request);
    +            final TableBuilder tableBuilder = new TableBuilder();
    +            MetaDataMutationResult result = mutateColumn(MutatateColumnType.MODIFY_COLUMN, metaData, new ColumnMutator() {
    +
    +                @Override
    +                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
    +                        List<Mutation> tableMetadata, Region region,
    +                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
    +                        long clientTimeStamp) throws IOException, SQLException {
    +
    +                    Preconditions.checkArgument(rowKeyMetaData.length == 5);
    +                    byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    +                    byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    +                    byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    +                    byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +
    +                    PColumn column = null;
    +                    Cell dataTypeCell = null;
    +                    Cell columnSizeCell = null;
    +                    Cell decimalDigitCell = null;
    +                    List<byte[]> mutatedTableNames = new ArrayList<>();
    +                    byte[] familyName = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
    +
    +                    try {
    +                        for (Mutation m : metaData) {
    +                            byte[][] rkmd = new byte[5][];
    +                            int pkCount = getVarChars(m.getRow(), rkmd);
    +
    +                            // Checking this put is for modifying a column
    +                            if (pkCount < COLUMN_NAME_INDEX || !(m instanceof Put)
    +                                    || rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX] == null) {
    +                                continue;
    +                            }
    +
    +                            List<Cell> cells;
    +                            if (rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null) {
    +                                PColumnFamily family = table.getColumnFamily(rkmd[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
    +                                column = family.getPColumnForColumnNameBytes(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
    +                                cells = m.getFamilyCellMap().get(column.getFamilyName().getBytes());
    +                            } else {
    +                                column = table.getPKColumn(new String(rkmd[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
    +                                cells = m.getFamilyCellMap().get(familyName);
    +                            }
    +
    +                            for (Cell cell : cells) {
    +                                if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DATA_TYPE_BYTES) == 0) {
    +                                    dataTypeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES) == 0) {
    +                                    columnSizeCell = cell;
    +                                } else if (Bytes.compareTo(CellUtil.cloneQualifier(cell), PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES) == 0) {
    +                                    decimalDigitCell = cell;
    +                                }
    +                            }
    +                        }
    +
    +                        // After PHOENIX-3534, we don't store parent table column metadata along with the child metadata,
    +                        // so we don't need to propagate changes to the child views.
    +
    +                        if (!table.getIndexes().isEmpty()) {
    +                            PhoenixConnection connection = null;
    +                            try {
    +                                connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                            } catch (ClassNotFoundException e) {
    +                            }
    +
    +                            for (PTable index : table.getIndexes()) {
    +                                byte[] tenantIdBytes = index.getTenantId() == null ?
    +                                                ByteUtil.EMPTY_BYTE_ARRAY :
    +                                                index.getTenantId().getBytes();
    +                                byte[] schemaNameBytes = index.getSchemaName().getBytes();
    +                                byte[] indexName = index.getTableName().getBytes();
    +                                byte[] indexKey = SchemaUtil.getTableKey(tenantIdBytes, schemaNameBytes, indexName);
    +
    +                                IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
    +                                boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(
    +                                                new Pair<>(column.getFamilyName().getString(), column.getName().getString()));
    +                                ColumnReference coveredColumn = indexMaintainer.getCoveredColumnsOfIndexTable(
    +                                                new ColumnReference(column.getFamilyName().getBytes(), column.getColumnQualifierBytes()));
    +
    +                                // Since the columns of fixed length which in present in primary key of index table will be converted
    +                                // to variable length when index tables are created, So we will not process indexed columns.
    +                                if (isColumnIndexed) {
    --- End diff --
    
    Ok


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224636819
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java ---
    @@ -242,19 +242,6 @@ public static Expression create(CompareOp op, List<Expression> children, Immutab
                       }
                     }
                 }
    -
    --- End diff --
    
    Why is this being removed?


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220413640
  
    --- Diff: phoenix-core/src/main/antlr3/PhoenixSQL.g ---
    @@ -655,9 +656,17 @@ alter_session_node returns [AlterSessionStatement ret]
     
     // Parse an alter table statement.
     alter_table_node returns [AlterTableStatement ret]
    -    :   ALTER (TABLE | v=VIEW) t=from_table_name
    -        ( (DROP COLUMN (IF ex=EXISTS)? c=column_names) | (ADD (IF NOT ex=EXISTS)? (d=column_defs) (p=fam_properties)?) | (SET (p=fam_properties)) )
    -        { PTableType tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW; ret = ( c == null ? factory.addColumn(factory.namedTable(null,t), tt, d, ex!=null, p) : factory.dropColumn(factory.namedTable(null,t), tt, c, ex!=null) ); }
    +@init { PTableType tt = null;}
    +    :   ALTER (TABLE | v=VIEW) t=from_table_name { tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW;}
    +            (
    +                DROP COLUMN (IF ex=EXISTS)? c=column_names { ret = factory.dropColumn(factory.namedTable(null, t), tt, c, ex!=null);}
    +                |
    +                    (
    +                        ADD (IF NOT ex=EXISTS)? cds=column_defs (p=fam_properties)?
    +                        | SET p=fam_properties
    +                     ) { ret = factory.addColumn(factory.namedTable(null, t), tt, cds, ex!=null, p);}
    +                | MODIFY (IF NOT ex=EXISTS)? cd=column_def { ret = factory.modifyColumn(factory.namedTable(null, t), tt, cd, ex!=null);}
    --- End diff --
    
    If the column does not exists and we try to modify it just throw an exception.


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r220420512
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3300,6 +3281,134 @@ private void mutateStringProperty(String tenantId, String schemaName, String tab
             }
         }
     
    +    public MutationState modifyColumn(ModifyColumnStatement statement) throws SQLException {
    +        PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
    +        ColumnDef columnDef = statement.getColumnDef();
    +        ColumnName columnName =columnDef.getColumnDefName();
    +
    +        // we can not modify index table/system table and project table.
    +        if (table.getType() != PTableType.TABLE && table.getType() != PTableType.VIEW) {
    +            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DISALLOW_MODIFY_TABLE_TYPE).build().buildException();
    +        }
    +
    +        PColumn oldColumn = null;
    +        for (PColumn column : table.getColumns()) {
    +            if (column.getFamilyName() == null) {
    +                if (column.getName().getString().equals(columnName.getColumnName())) {
    +                    oldColumn = column;
    +                }
    +            } else {
    +                if (column.getName().getString().equals(columnName.getColumnName()) &&
    +                        ((columnName.getFamilyName() != null && column.getFamilyName().getString().equals(columnName.getFamilyName())) ||
    +                                (columnName.getFamilyName() == null && column.getFamilyName().getString().equals(QueryConstants.DEFAULT_COLUMN_FAMILY)))) {
    +                    oldColumn = column;
    +                    break;
    +                }
    +            }
    +        }
    +
    +        if (oldColumn == null) {
    +            throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(),
    +                    columnName.getFamilyName(), columnName.getColumnName());
    +        }
    +
    +        // Comparision of row keys were affected when we changed max length of pk columns to pad more placeholder,
    +        // so we can not modify length of the PK column.
    +        if (oldColumn.isRowTimestamp() || SchemaUtil.isPKColumn(oldColumn)) {
    --- End diff --
    
    What happens if you are changing a column that is present in the primary key of an index table?


---

[GitHub] phoenix pull request #355: support alter modify column grammar

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/355#discussion_r224636783
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java ---
    @@ -392,6 +375,13 @@ public SQLException newException(SQLExceptionInfo info) {
         INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES(1137, "XCL37", "If IMMUTABLE_STORAGE_SCHEME property is not set to ONE_CELL_PER_COLUMN COLUMN_ENCODED_BYTES cannot be 0"),
         INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE(1138, "XCL38", "IMMUTABLE_STORAGE_SCHEME property cannot be changed from/to ONE_CELL_PER_COLUMN "),
         CANNOT_SET_GUIDE_POST_WIDTH(1139, "XCL39", "Guide post width can only be set on base data tables"),
    +
    +    INVALID_DATA_TYPE_CAST(1141, "XCL41", "Can't modify column because of invalid data type cast"),
    +    DISALLOW_MODIFY_TABLE_TYPE(1142, "XCL42", "Only common table or view can be modified!"),
    +    DISALLOW_MODIFY_TIMESTAMP_OR_PK_COLUMN(1143, "XCL43", "Timestamp column or PK column can not be modified!"),
    +    DISALLOW_MODIFY_CHILD_TABLE(1144, "XCL44", "This table have a parent table which can not be modified!"),
    +    DISALLOW_DECREASE_CHAR_LENGTH(1145, "XCL45", "Can't decrease length of char type which less than the old data type"),
    --- End diff --
    
    We should prevent decreasing the length of any data type.


---