You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by twdsilva <gi...@git.apache.org> on 2018/08/17 01:06:52 UTC

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

GitHub user twdsilva opened a pull request:

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

    PHOENIX-4798 Update encoded col qualifiers on the base table correctly

    @ChinmaySKulkarni  @karanmehta93 Can you please review?
    This doc describes how the client side mutexes are used
    https://docs.google.com/spreadsheets/d/1EOkuIdp4_OHg-l-ArdZYnAkMiCreDUIBjGXzdlJHW2w/edit?usp=sharing

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

    $ git pull https://github.com/twdsilva/phoenix PHOENIX-4798

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

    https://github.com/apache/phoenix/pull/329.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 #329
    
----
commit c9e5d0ef56d5600bb1cadafc01429bea5f364623
Author: Thomas D'Silva <td...@...>
Date:   2018-08-15T19:23:56Z

    PHOENIX-4798 Update encoded col qualifiers on the base table correctly

----


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211060525
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3207,17 +3226,37 @@ public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaDa
                         } 
                         if (type == PTableType.VIEW
                                 && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    -                        /*
    -                         * When adding a column to a view that uses encoded column name scheme, we
    -                         * need to modify the CQ counters stored in the view's physical table. So to
    -                         * make sure clients get the latest PTable, we need to invalidate the cache
    -                         * entry.
    -                         */
    -                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
    -                                .getPhysicalTableRowForView(table)));
    -
    -
    -
    +                                /*
    +                                 * When adding a column to a view that uses encoded column name
    +                                 * scheme, we need to modify the CQ counters stored in the view's
    +                                 * physical table. So to make sure clients get the latest PTable, we
    +                                 * need to invalidate the cache entry.
    +                                 */
    +                                invalidateList.add(new ImmutableBytesPtr(
    +                                        MetaDataUtil.getPhysicalTableRowForView(table)));
    +                                byte[] parentSchemaName =
    +                                        SchemaUtil.getSchemaNameFromFullName(
    +                                            table.getPhysicalName().getString()).getBytes();
    +                                byte[] parentTableName =
    --- End diff --
    
    PTable.getParentTableName and PTable.getParentSchemaName() can be used here.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211060867
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2231,6 +2231,7 @@ public void createTable(RpcController controller, CreateTableRequest request,
                                         PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
                                         remoteMutations, fullTableName,
                                         MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
    +                        clearParentTableFromCache(clientTimeStamp, table);
    --- End diff --
    
    Why explicitly we need to clear the cache, doesn't the server cache holds the current table always?


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211435859
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3207,17 +3226,37 @@ public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaDa
                         } 
                         if (type == PTableType.VIEW
                                 && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    -                        /*
    -                         * When adding a column to a view that uses encoded column name scheme, we
    -                         * need to modify the CQ counters stored in the view's physical table. So to
    -                         * make sure clients get the latest PTable, we need to invalidate the cache
    -                         * entry.
    -                         */
    -                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
    -                                .getPhysicalTableRowForView(table)));
    -
    -
    -
    +                                /*
    +                                 * When adding a column to a view that uses encoded column name
    +                                 * scheme, we need to modify the CQ counters stored in the view's
    +                                 * physical table. So to make sure clients get the latest PTable, we
    +                                 * need to invalidate the cache entry.
    +                                 */
    +                                invalidateList.add(new ImmutableBytesPtr(
    +                                        MetaDataUtil.getPhysicalTableRowForView(table)));
    +                                byte[] parentSchemaName =
    +                                        SchemaUtil.getSchemaNameFromFullName(
    +                                            table.getPhysicalName().getString()).getBytes();
    +                                byte[] parentTableName =
    --- End diff --
    
    done, I should have used the parent table instead of the view table here


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211447479
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3681,7 +3691,12 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    -            if (!acquiredColumnMutexSet.isEmpty()) {
    +            if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    +                // release the mutex on the physical table (used to prevent concurrent conflicting
    +                // add column changes)
    +                deleteCell(null, physicalSchemaName, physicalTableName, null);
    --- End diff --
    
    done, I missed that.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211060422
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3681,7 +3691,12 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    -            if (!acquiredColumnMutexSet.isEmpty()) {
    +            if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    +                // release the mutex on the physical table (used to prevent concurrent conflicting
    +                // add column changes)
    +                deleteCell(null, physicalSchemaName, physicalTableName, null);
    --- End diff --
    
    should we check here that we have acquired the mutex?


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r212054923
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3291,6 +3305,29 @@ public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaDa
                             }
                         }
                         tableMetaData.addAll(additionalTableMetadataMutations);
    +                    if (type == PTableType.VIEW
    +                                && EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol
    +                                && !table.isAppendOnlySchema()) {
    +                                // When adding a column to a view that uses encoded column name
    +                                // scheme, we need to modify the CQ counters stored in the view's
    +                                // physical table. So to make sure clients get the latest PTable, we
    +                                // need to invalidate the cache entry.
    +                                // If the table uses APPEND_ONLY_SCHEMA we use the position of the
    +                                // column as the encoded column qualifier and so we don't need to
    +                                // update the CQ counter in the view physical table (see
    +                                // PHOENIX-4737)
    +                                invalidateList.add(new ImmutableBytesPtr(
    +                                        MetaDataUtil.getPhysicalTableRowForView(table)));
    +                                byte[] parentSchemaName =
    +                                        table.getParentSchemaName() != null
    +                                                ? table.getParentSchemaName().getBytes()
    +                                                : ByteUtil.EMPTY_BYTE_ARRAY;
    +                                byte[] parentTableName = table.getParentTableName().getBytes();
    +                                PTable parentTable =
    +                                        doGetTable(null, parentSchemaName, parentTableName,
    +                                            clientTimeStamp, null, request.getClientVersion(), true,
    +                                            true, null);
    --- End diff --
    
    You may need to remove doGetTable call as parentTable is not used anywhere now?


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211797286
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java ---
    @@ -303,6 +303,54 @@ public static long getSequenceNumber(List<Mutation> tableMetaData) {
             return getSequenceNumber(getPutOnlyTableHeaderRow(tableMetaData));
         }
         
    +    /**
    +     * Returns the sequence number of the parent table if we have a mutation that is updating the
    +     * parent table's encoded column qualifier
    +     */
    +    public static long getParentSequenceNumberForAddColumnMutations(List<Mutation> tableMetaData) {
    +        byte[] parentTableRowKey = null;
    +        for (Mutation tableMutation : tableMetaData) {
    --- End diff --
    
    I don't think we really need this.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211801654
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java ---
    @@ -303,6 +303,54 @@ public static long getSequenceNumber(List<Mutation> tableMetaData) {
             return getSequenceNumber(getPutOnlyTableHeaderRow(tableMetaData));
         }
         
    +    /**
    +     * Returns the sequence number of the parent table if we have a mutation that is updating the
    +     * parent table's encoded column qualifier
    +     */
    +    public static long getParentSequenceNumberForAddColumnMutations(List<Mutation> tableMetaData) {
    +        byte[] parentTableRowKey = null;
    +        for (Mutation tableMutation : tableMetaData) {
    --- End diff --
    
    Done. We don't need this because we acquire a mutex on the client side which will prevent concurrent changes to the encoded col qualifier counter. 


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211434951
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2231,6 +2231,7 @@ public void createTable(RpcController controller, CreateTableRequest request,
                                         PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
                                         remoteMutations, fullTableName,
                                         MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
    +                        clearParentTableFromCache(clientTimeStamp, table);
    --- End diff --
    
    Since we are using the HTable api to update the encoded col qualifiers and the sequence number of the parent we need to remove it from the cache of the region server on which it lives so that the next time the parent tableis resolved the latest metadata gets loaded. 


---

[GitHub] phoenix issue #329: PHOENIX-4798 Update encoded col qualifiers on the base t...

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

    https://github.com/apache/phoenix/pull/329
  
    @ankitsinghal  Thanks for the review, I made the change and pushed to 4.x and master branches.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211447285
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java ---
    @@ -303,6 +303,54 @@ public static long getSequenceNumber(List<Mutation> tableMetaData) {
             return getSequenceNumber(getPutOnlyTableHeaderRow(tableMetaData));
         }
         
    +    /**
    +     * Returns the sequence number of the parent table if we have a mutation that is updating the
    +     * parent table's encoded column qualifier
    +     */
    +    public static long getParentSequenceNumberForAddColumnMutations(List<Mutation> tableMetaData) {
    +        byte[] parentTableRowKey = null;
    +        for (Mutation tableMutation : tableMetaData) {
    +            List<Cell> kvs =
    +                    tableMutation.getFamilyCellMap()
    +                            .get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
    +            if (kvs != null) {
    +                for (Cell kv : kvs) {
    +                    if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
    +                        kv.getQualifierLength(), PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES, 0,
    +                        PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES.length) == 0) {
    +                        parentTableRowKey = kv.getRow();
    +                    }
    +                }
    +            }
    +        }
    +        if (parentTableRowKey == null) {
    +            throw new IllegalStateException(
    +                    "Could not find mutation to update the encoded column qualifier of the parent table ");
    +        }
    +        
    +        byte[] rowKeyPrefix = ByteUtil.concat(parentTableRowKey, ByteUtil.EMPTY_BYTE_ARRAY);
    --- End diff --
    
    Sorry I had a bug here in the first loop we should look for COLUMN_QUALIFIER_COUNTER_BYTES. 
    tableMetadata will contain two Puts that have TABLE_SEQ_NUM, one is for the view where we update the COLUMN_COUNT and another is for the physical table where we update the COLUMN_QUALIFIER_COUNTER. I think I could assume the second put will always be the one for the physical table, but I added this check to search for the keyvalue of the COLUMN_QUALIFIER_COUNTER put just to be more resilient.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211437117
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3207,17 +3226,37 @@ public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaDa
                         } 
                         if (type == PTableType.VIEW
                                 && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    -                        /*
    -                         * When adding a column to a view that uses encoded column name scheme, we
    -                         * need to modify the CQ counters stored in the view's physical table. So to
    -                         * make sure clients get the latest PTable, we need to invalidate the cache
    -                         * entry.
    -                         */
    -                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
    -                                .getPhysicalTableRowForView(table)));
    -
    -
    -
    +                                /*
    +                                 * When adding a column to a view that uses encoded column name
    +                                 * scheme, we need to modify the CQ counters stored in the view's
    +                                 * physical table. So to make sure clients get the latest PTable, we
    +                                 * need to invalidate the cache entry.
    +                                 */
    +                                invalidateList.add(new ImmutableBytesPtr(
    +                                        MetaDataUtil.getPhysicalTableRowForView(table)));
    +                                byte[] parentSchemaName =
    +                                        SchemaUtil.getSchemaNameFromFullName(
    +                                            table.getPhysicalName().getString()).getBytes();
    +                                byte[] parentTableName =
    +                                        SchemaUtil
    +                                                .getTableNameFromFullName(
    +                                                    (table.getPhysicalName().getString()))
    +                                                .getBytes();
    +                                PTable parentTable =
    +                                        doGetTable(null, parentSchemaName, parentTableName,
    --- End diff --
    
    As part of PHOENIX-3534 I modified doGetTable() to check if the table being looked up is on the current region, if its not it looks up the table using queryServices.getTable()


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211060797
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3207,17 +3226,37 @@ public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaDa
                         } 
                         if (type == PTableType.VIEW
                                 && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
    -                        /*
    -                         * When adding a column to a view that uses encoded column name scheme, we
    -                         * need to modify the CQ counters stored in the view's physical table. So to
    -                         * make sure clients get the latest PTable, we need to invalidate the cache
    -                         * entry.
    -                         */
    -                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
    -                                .getPhysicalTableRowForView(table)));
    -
    -
    -
    +                                /*
    +                                 * When adding a column to a view that uses encoded column name
    +                                 * scheme, we need to modify the CQ counters stored in the view's
    +                                 * physical table. So to make sure clients get the latest PTable, we
    +                                 * need to invalidate the cache entry.
    +                                 */
    +                                invalidateList.add(new ImmutableBytesPtr(
    +                                        MetaDataUtil.getPhysicalTableRowForView(table)));
    +                                byte[] parentSchemaName =
    +                                        SchemaUtil.getSchemaNameFromFullName(
    +                                            table.getPhysicalName().getString()).getBytes();
    +                                byte[] parentTableName =
    +                                        SchemaUtil
    +                                                .getTableNameFromFullName(
    +                                                    (table.getPhysicalName().getString()))
    +                                                .getBytes();
    +                                PTable parentTable =
    +                                        doGetTable(null, parentSchemaName, parentTableName,
    --- End diff --
    
    doGetTable expects table key to be in the same region, so if a view is created in different schema than the parent schema, will it work in case of mult-region SYSTEM.CATALOG table.


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva closed the pull request at:

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


---

[GitHub] phoenix issue #329: PHOENIX-4798 Update encoded col qualifiers on the base t...

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

    https://github.com/apache/phoenix/pull/329
  
    @ankitsinghal , could you please review?


---

[GitHub] phoenix pull request #329: PHOENIX-4798 Update encoded col qualifiers on the...

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

    https://github.com/apache/phoenix/pull/329#discussion_r211060359
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java ---
    @@ -303,6 +303,54 @@ public static long getSequenceNumber(List<Mutation> tableMetaData) {
             return getSequenceNumber(getPutOnlyTableHeaderRow(tableMetaData));
         }
         
    +    /**
    +     * Returns the sequence number of the parent table if we have a mutation that is updating the
    +     * parent table's encoded column qualifier
    +     */
    +    public static long getParentSequenceNumberForAddColumnMutations(List<Mutation> tableMetaData) {
    +        byte[] parentTableRowKey = null;
    +        for (Mutation tableMutation : tableMetaData) {
    +            List<Cell> kvs =
    +                    tableMutation.getFamilyCellMap()
    +                            .get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
    +            if (kvs != null) {
    +                for (Cell kv : kvs) {
    +                    if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
    +                        kv.getQualifierLength(), PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES, 0,
    +                        PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES.length) == 0) {
    +                        parentTableRowKey = kv.getRow();
    +                    }
    +                }
    +            }
    +        }
    +        if (parentTableRowKey == null) {
    +            throw new IllegalStateException(
    +                    "Could not find mutation to update the encoded column qualifier of the parent table ");
    +        }
    +        
    +        byte[] rowKeyPrefix = ByteUtil.concat(parentTableRowKey, ByteUtil.EMPTY_BYTE_ARRAY);
    --- End diff --
    
    wouldn't be rowKeyPrefix and parentTableRowkey equal here and so why two separate loops? is it a typo error for SEPARATOR_BYTE?


---