You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by twdsilva <gi...@git.apache.org> on 2018/07/24 18:00:42 UTC

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

GitHub user twdsilva opened a pull request:

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

    PHOENIX-4799 Write cells using checkAndMutate to prevent conflicting …

    …changes
    
    @vincentpoon @karanmehta93 @ChinmaySKulkarni  Can you please review? 
    With this patch, when we add a column we write a cell to SYSTEM.CHILD_LINK with row key (tenantId, schemaName, physicalTableName, columnName) to prevent conflicting concurrent modifications. 
    While dropping a table or creating a view we also write a cell to SYSTEM.CHILD_LINK with row key (tenantId, schemaName, physicalTableName). 
    This is done in MetadataClient before we make an rpc to create view / drop table / add column. 

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

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

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

    https://github.com/apache/phoenix/pull/313.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 #313
    
----
commit 421888f914db3307906aa4424e24e202834b3e08
Author: Thomas D'Silva <td...@...>
Date:   2018-07-24T17:49:31Z

    PHOENIX-4799 Write cells using checkAndMutate to prevent conflicting changes

----


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    @ankitsinghal  I filed PHOENIX-4765 to not allow metadata changes on a base table that has child views when the request is from an older client. This will also allow us to rollback the upgraded server side jar if required. 


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    @ankitsinghal  Thanks for the review. I have updated the PR to based on the feedback. Can you please take a look?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206330492
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3604,6 +3681,18 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            if (acquiredMutex && !columns.isEmpty()) {
    --- End diff --
    
    where is acquiredMutex set to true?
    



---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r205594858
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3604,6 +3675,17 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            if (!columns.isEmpty()) {
    +                for (PColumn pColumn : columns) {
    +                    PName physicalName = table.getPhysicalName();
    +                    String physicalSchemaName =
    +                            SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
    +                    String physicalTableName =
    +                            SchemaUtil.getTableNameFromFullName(physicalName.getString());
    +                    deleteCell(null, physicalSchemaName, physicalTableName,
    --- End diff --
    
    @twdsilva , shouldn't you check whether you have acquired a lock(or inserted the cell in Mutex) before deleting the lock cell, because here you might be deleting the locks acquired by some other threads. And, You may need to do this per column I think.


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    Testing issues@phoenix.apache.org.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r207674894
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -2957,6 +3006,16 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                     Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                     tableMetaData.add(linkDelete);
                 }
    +            if (tableType == PTableType.TABLE) {
    +                // acquire a mutex on the table to prevent creating views while concurrently
    +                // dropping the base table
    +                acquiredMutex = writeCell(null, schemaName, tableName, null);
    +                if (!acquiredMutex) {
    +                    logger.info("Failed to acquire mutex on physical table " + physicalTableName);
    --- End diff --
    
    nit: You can push mutex related logs lines to the specific methods.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r204968948
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -1874,6 +1881,31 @@ private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, Primary
             }
             return false;
         }
    +    
    +	/**
    +	 * If we are creating a view we write a cell to the SYSTEM.MUTEX table
    +	 * with the rowkey of the parent table to prevent concurrent modifications
    +	 */
    +	private void writeCell(String tenantId, String schemaName, String tableName, String columnName) throws SQLException {
    +		byte[] rowKey = columnName != null
    +				? SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName, null)
    +				: SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    +		boolean success = connection.getQueryServices().writeMutexCell(rowKey);
    +		if (!success) {
    +			throw new ConcurrentTableMutationException(schemaName, tableName);
    +		}
    +	}
    +	
    +	/**
    +	 * Remove the cell that was written to to the SYSTEM.CHILD_LINK table with
    --- End diff --
    
    Update the comment since you're using SYSMUTEX now


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206687937
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -2957,6 +3010,11 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                     Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                     tableMetaData.add(linkDelete);
                 }
    +            if (tableType == PTableType.TABLE) {
    +                // acquire a mutex on the table to prevent creating views while concurrently
    +                // dropping the base table
    +                acquiredMutex = writeCell(null, schemaName, tableName, null);
    --- End diff --
    
    Yes, I missed that. I originally had writeCell throw the ConcurrentTableMutationException, but we need to know we need writeCell to return false if it wasn't able to do the checkAndPut so that we can set the acquiredMutex boolean which is later used to determine if we have to delete the cell in the finally block.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206415547
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -1913,6 +1940,21 @@ private PTable createTableInternal(CreateTableStatement statement, byte[][] spli
                 boolean isLocalIndex = indexType == IndexType.LOCAL;
                 QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
                 ImmutableStorageScheme immutableStorageScheme = ONE_CELL_PER_COLUMN;
    +            
    +            if (tableType == PTableType.VIEW) {
    --- End diff --
    
    nit: Would be good to have some logs here for debugging purposes.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

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


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206738423
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -2723,13 +2712,10 @@ private void createOtherSystemTables(PhoenixConnection metaConnection, HBaseAdmi
             try {
                 metaConnection.createStatement().executeUpdate(getChildLinkDDL());
             } catch (TableAlreadyExistsException e) {}
    -        // Catch the IOException to log the error message and then bubble it up for the client to retry.
             try {
    -            createSysMutexTableIfNotExists(hbaseAdmin);
    -        } catch (IOException exception) {
    -            logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
    -            throw exception;
    -        }
    +            metaConnection.createStatement().executeUpdate(getMutexDDL());
    --- End diff --
    
    ok, then we are good.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206414686
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -3410,17 +3405,10 @@ void ensureSystemTablesMigratedToSystemNamespace()
                 // No tables exist matching "SYSTEM\..*", they are all already in "SYSTEM:.*"
                 if (tableNames.size() == 0) { return; }
                 // Try to move any remaining tables matching "SYSTEM\..*" into "SYSTEM:"
    -            if (tableNames.size() > 5) {
    +            if (tableNames.size() > 7) {
                     logger.warn("Expected 5 system tables but found " + tableNames.size() + ":" + tableNames);
    --- End diff --
    
    nit: change log message appropriately. Extract the constant to a variable


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    @ankitsinghal  @karanmehta93  I updated the PR with the latest feedback, can you please review?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r204968823
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java ---
    @@ -123,6 +148,48 @@ public static void doSetup() throws Exception {
             }
         }
         
    +	public static class TestMetaDataRegionObserver extends BaseRegionObserver {
    +		@Override
    +		public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
    +				MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
    +			if (shouldFail(c, miniBatchOp.getOperation(0))) {
    +				// throwing anything other than instances of IOException result
    +				// in this coprocessor being unloaded
    +				// DoNotRetryIOException tells HBase not to retry this mutation
    +				// multiple times
    +				throw new DoNotRetryIOException();
    +			} else if (shouldSlowDown(c, miniBatchOp.getOperation(0))) {
    +				// simulate a slow write to SYSTEM.CATALOG
    --- End diff --
    
    Nice!


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206738342
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -355,5 +355,19 @@
     			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
     			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
    +	
    +	 public static final String CREATE_MUTEX_METADTA =
    +	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +	            // Pk columns
    +	            TENANT_ID + " VARCHAR NULL," +
    +	            TABLE_SCHEM + " VARCHAR NULL," +
    +	            TABLE_NAME + " VARCHAR NOT NULL," +
    +	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	            HConstants.VERSIONS + "=%s,\n" +
    +	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
    +	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    --- End diff --
    
    ok make sense, so can you just update QueryServices.writeMutexCell(byte[] rowKey) and deleteMutexCell(byte[] rowKey) to accept all the arguments which form the right primary key for the table for consistency. 


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r205595041
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3034,6 +3088,11 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                 return new MutationState(0, 0, connection);
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            // acquire a mutex on the table to prevent creating views while concurrently
    +            // dropping the base table
    +            if (tableType == PTableType.TABLE) {
    +                deleteCell(null, schemaName, tableName, null);
    +            }
    --- End diff --
    
    nit: Please update the comment here that you are releasing a mutex.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r207681657
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -2957,6 +3006,16 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                     Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                     tableMetaData.add(linkDelete);
                 }
    +            if (tableType == PTableType.TABLE) {
    +                // acquire a mutex on the table to prevent creating views while concurrently
    +                // dropping the base table
    +                acquiredMutex = writeCell(null, schemaName, tableName, null);
    +                if (!acquiredMutex) {
    +                    logger.info("Failed to acquire mutex on physical table " + physicalTableName);
    --- End diff --
    
    Done, can I get a +1?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206737372
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -2723,13 +2712,10 @@ private void createOtherSystemTables(PhoenixConnection metaConnection, HBaseAdmi
             try {
                 metaConnection.createStatement().executeUpdate(getChildLinkDDL());
             } catch (TableAlreadyExistsException e) {}
    -        // Catch the IOException to log the error message and then bubble it up for the client to retry.
             try {
    -            createSysMutexTableIfNotExists(hbaseAdmin);
    -        } catch (IOException exception) {
    -            logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
    -            throw exception;
    -        }
    +            metaConnection.createStatement().executeUpdate(getMutexDDL());
    --- End diff --
    
    The mutex table would have been created by createSysMutexTableIfNotExists(), we call execute the CREATE TABLE statement so that it exists in SYSTEM.CATALOG, so that we can use GRANT/REVOKE to grant permission on the SYSTEM.MUTEX table.


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    We can't add/drop columns directly to an index on a view. Views with their own child views will write a mutex using the physical table.
    
    @ChinmaySKulkarni and @karanmehta93  thanks for the feedback. I have updated the PR, please take a look.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206411967
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3604,6 +3681,18 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            if (acquiredMutex && !columns.isEmpty()) {
    --- End diff --
    
    @ankitsinghal  Sorry, I didn't update the PR with all my latest changes, I fixed this. Can you please review?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206735726
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -355,5 +355,19 @@
     			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
     			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
    +	
    +	 public static final String CREATE_MUTEX_METADTA =
    +	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +	            // Pk columns
    +	            TENANT_ID + " VARCHAR NULL," +
    +	            TABLE_SCHEM + " VARCHAR NULL," +
    +	            TABLE_NAME + " VARCHAR NOT NULL," +
    +	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	            HConstants.VERSIONS + "=%s,\n" +
    +	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
    +	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    --- End diff --
    
    Why there is a need of creating a Phoenix managed table for mutex? 
    And also API in QueryServices.writeMutexCell(byte[] rowKey) and deleteMutexCell(byte[] rowKey) don't enforce the schema of the table will be followed. 



---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    Thanks for the suggestion, I will update the patch to use SYSTEM.MUTEX


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206415970
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -2957,6 +3010,11 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                     Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                     tableMetaData.add(linkDelete);
                 }
    +            if (tableType == PTableType.TABLE) {
    +                // acquire a mutex on the table to prevent creating views while concurrently
    +                // dropping the base table
    +                acquiredMutex = writeCell(null, schemaName, tableName, null);
    --- End diff --
    
    Nothing is prevent even if `acquiredMutex` is false here. I think you should throw `ConcurrentTableMutationException` here as well? You might might wanna push down that logic to `writeCell()` method?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206687568
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -1913,6 +1940,21 @@ private PTable createTableInternal(CreateTableStatement statement, byte[][] spli
                 boolean isLocalIndex = indexType == IndexType.LOCAL;
                 QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
                 ImmutableStorageScheme immutableStorageScheme = ONE_CELL_PER_COLUMN;
    +            
    +            if (tableType == PTableType.VIEW) {
    --- End diff --
    
    Done


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    A few comments and nits. Do we have to do anything special to handle indexes on views? I guess a view on a view should be fine with this implementation since the physical table name would resolve to the final base table. @twdsilva 


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206741679
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -355,5 +355,19 @@
     			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
     			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
    +	
    +	 public static final String CREATE_MUTEX_METADTA =
    +	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +	            // Pk columns
    +	            TENANT_ID + " VARCHAR NULL," +
    +	            TABLE_SCHEM + " VARCHAR NULL," +
    +	            TABLE_NAME + " VARCHAR NOT NULL," +
    +	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	            HConstants.VERSIONS + "=%s,\n" +
    +	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
    +	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    --- End diff --
    
    Yes, probably our comments have crossed.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r205950165
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3604,6 +3675,17 @@ public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
                 }
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            if (!columns.isEmpty()) {
    +                for (PColumn pColumn : columns) {
    +                    PName physicalName = table.getPhysicalName();
    +                    String physicalSchemaName =
    +                            SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
    +                    String physicalTableName =
    +                            SchemaUtil.getTableNameFromFullName(physicalName.getString());
    +                    deleteCell(null, physicalSchemaName, physicalTableName,
    --- End diff --
    
    I changed the code to only delete the cells if we were able to successfully do the checkAndPut. 
    When we add a column we write a cell per column that we are creating. 
    When we drop a base table or create a view, we write a single cell with the rowkey of the physical table (to prevent a view being created while we are dropping the base table).


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206736061
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -2723,13 +2712,10 @@ private void createOtherSystemTables(PhoenixConnection metaConnection, HBaseAdmi
             try {
                 metaConnection.createStatement().executeUpdate(getChildLinkDDL());
             } catch (TableAlreadyExistsException e) {}
    -        // Catch the IOException to log the error message and then bubble it up for the client to retry.
             try {
    -            createSysMutexTableIfNotExists(hbaseAdmin);
    -        } catch (IOException exception) {
    -            logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
    -            throw exception;
    -        }
    +            metaConnection.createStatement().executeUpdate(getMutexDDL());
    --- End diff --
    
    Shouldn't the mutex table available and have acquired a mutex already for the upgrade before you call createOtherSystemTables


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    @karanmehta93  I have updated the PR to use SYSTEM.MUTEX, please review.


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    @karanmehta93 or @ankitsinghal  Can you please review the updated PR, thanks!


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206687510
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -16,8 +16,29 @@
      */
     package org.apache.phoenix.end2end;
     
    -import com.google.common.base.Joiner;
    -import com.google.common.base.Throwables;
    +import static org.junit.Assert.assertEquals;
    --- End diff --
    
    I used the dev/phoenix.importorder format, so this should be fine.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r204968683
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java ---
    @@ -1274,28 +1341,145 @@ public void testChildViewCreationFails() throws Exception {
             PhoenixRuntime.getTableNoCache(conn, fullViewName2);
         }
         
    -    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
    -    private static final byte[] ROWKEY_TO_FAIL_BYTES = SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
    -            Bytes.toBytes(FAILED_VIEWNAME));
    -    
    -    public static class FailingRegionObserver extends SimpleRegionObserver {
    -        @Override
    -        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
    -                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
    -            if (shouldFail(c, miniBatchOp.getOperation(0))) {
    -                // throwing anything other than instances of IOException result
    -                // in this coprocessor being unloaded
    -                // DoNotRetryIOException tells HBase not to retry this mutation
    -                // multiple times
    -                throw new DoNotRetryIOException();
    -            }
    -        }
    -
    -        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
    -            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
    -            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
    -                    && (Bytes.equals(ROWKEY_TO_FAIL_BYTES, m.getRow()));
    -        }
    -
    -    }
    +	@Test
    +	public void testConcurrentViewCreationAndTableDrop() throws Exception {
    +		try (Connection conn = DriverManager.getConnection(getUrl())) {
    +			String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +			String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
    +			String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
    +			latch1 = new CountDownLatch(1);
    +			latch2 = new CountDownLatch(1);
    +			String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
    +					+ tableDDLOptions;
    +			conn.createStatement().execute(tableDdl);
    +
    +			ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
    +				@Override
    +				public Thread newThread(Runnable r) {
    +					Thread t = Executors.defaultThreadFactory().newThread(r);
    +					t.setDaemon(true);
    +					t.setPriority(Thread.MIN_PRIORITY);
    +					return t;
    +				}
    +			});
    +
    +			// create the view in a separate thread (which will take some time
    +			// to complete)
    +			Future<Exception> future = executorService.submit(new CreateViewRunnable(fullTableName, fullViewName1));
    +			// wait till the thread makes the rpc to create the view
    +			latch1.await();
    +			tableDdl = "DROP TABLE " + fullTableName;
    +			try {
    +				// drop table should fail as we are concurrently adding a view
    +				conn.createStatement().execute(tableDdl);
    +				fail("Creating a view while concurrently dropping the base table should fail");
    +			} catch (ConcurrentTableMutationException e) {
    +			}
    +			latch2.countDown();
    +
    +			Exception e = future.get();
    +			assertTrue(e == null);
    +
    +			// create another view to ensure that the cell used to prevent
    +			// concurrent modifications was removed
    +			String ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
    +					+ " WHERE k = 6";
    +			conn.createStatement().execute(ddl);
    +		}
    +	}
    +
    +	@Test
    +	public void testConcurrentAddColumn() throws Exception {
    +		try (Connection conn = DriverManager.getConnection(getUrl())) {
    +			String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +			String fullViewName = SchemaUtil.getTableName(SCHEMA2, SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
    +			// create base table
    +			String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
    +					+ tableDDLOptions;
    +			conn.createStatement().execute(tableDdl);
    +			// create a view
    +			String ddl = "CREATE VIEW " + fullViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
    +					+ " WHERE k = 6";
    +			conn.createStatement().execute(ddl);
    +
    +			latch1 = new CountDownLatch(1);
    +			latch2 = new CountDownLatch(1);
    +			ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
    +				@Override
    +				public Thread newThread(Runnable r) {
    +					Thread t = Executors.defaultThreadFactory().newThread(r);
    +					t.setDaemon(true);
    +					t.setPriority(Thread.MIN_PRIORITY);
    +					return t;
    +				}
    +			});
    +
    +			// add a column to the view in a separate thread (which will take
    +			// some time to complete)
    +			Future<Exception> future = executorService.submit(new AddColumnRunnable(fullViewName));
    +			// wait till the thread makes the rpc to create the view
    +			boolean result = latch1.await(2, TimeUnit.MINUTES);
    +			if (!result) {
    +				fail("The create view rpc look too long");
    +			}
    +			tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
    +			try {
    +				// add the same column to the base table with a different type
    +				conn.createStatement().execute(tableDdl);
    +				fail("Creating a view while concurrently dropping the base table should fail");
    +			} catch (ConcurrentTableMutationException e) {
    +			}
    +			latch2.countDown();
    +
    +			Exception e = future.get();
    +			assertTrue(e == null);
    +			
    +			// add a new column to the base table to ensure that the cell used
    +			// to prevent concurrent modifications was removed
    +			tableDdl = "ALTER TABLE " + fullTableName + " ADD v4 INTEGER";
    --- End diff --
    
    You didn't execute this ddl statement


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206414419
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -2723,6 +2712,9 @@ private void createOtherSystemTables(PhoenixConnection metaConnection, HBaseAdmi
             try {
                 metaConnection.createStatement().executeUpdate(getChildLinkDDL());
             } catch (TableAlreadyExistsException e) {}
    +        try {
    +            metaConnection.createStatement().executeUpdate(getMutexDDL());
    +        } catch (TableAlreadyExistsException e) {}
             // Catch the IOException to log the error message and then bubble it up for the client to retry.
             try {
                 createSysMutexTableIfNotExists(hbaseAdmin);
    --- End diff --
    
    Do we still need this part of code?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206412733
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -82,6 +82,10 @@
         static final String SYSTEM_SEQUENCE_IDENTIFIER =
                 QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
     
    +    static final String SYSTEM_MUTEX_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\""
    +                    + PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME + "\"";
    +
         static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    --- End diff --
    
    Do you need to SYSTEM:MUTEX over here as well since now its visible from Phoenix level as well?


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206737111
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -355,5 +355,19 @@
     			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
     			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
    +	
    +	 public static final String CREATE_MUTEX_METADTA =
    +	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +	            // Pk columns
    +	            TENANT_ID + " VARCHAR NULL," +
    +	            TABLE_SCHEM + " VARCHAR NULL," +
    +	            TABLE_NAME + " VARCHAR NOT NULL," +
    +	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	            HConstants.VERSIONS + "=%s,\n" +
    +	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
    +	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    --- End diff --
    
    For clusters that use namespace mapping and that map the phoenix system tables to the SYSTEM namespace, we want to be able to use the GRANT/REVOKE statements to grant RW access to the SYSTEM:MUTEX table.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r204968699
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java ---
    @@ -1274,28 +1341,145 @@ public void testChildViewCreationFails() throws Exception {
             PhoenixRuntime.getTableNoCache(conn, fullViewName2);
         }
         
    -    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
    -    private static final byte[] ROWKEY_TO_FAIL_BYTES = SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
    -            Bytes.toBytes(FAILED_VIEWNAME));
    -    
    -    public static class FailingRegionObserver extends SimpleRegionObserver {
    -        @Override
    -        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
    -                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
    -            if (shouldFail(c, miniBatchOp.getOperation(0))) {
    -                // throwing anything other than instances of IOException result
    -                // in this coprocessor being unloaded
    -                // DoNotRetryIOException tells HBase not to retry this mutation
    -                // multiple times
    -                throw new DoNotRetryIOException();
    -            }
    -        }
    -
    -        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
    -            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
    -            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
    -                    && (Bytes.equals(ROWKEY_TO_FAIL_BYTES, m.getRow()));
    -        }
    -
    -    }
    +	@Test
    +	public void testConcurrentViewCreationAndTableDrop() throws Exception {
    +		try (Connection conn = DriverManager.getConnection(getUrl())) {
    +			String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +			String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
    +			String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
    +			latch1 = new CountDownLatch(1);
    +			latch2 = new CountDownLatch(1);
    +			String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
    +					+ tableDDLOptions;
    +			conn.createStatement().execute(tableDdl);
    +
    +			ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
    +				@Override
    +				public Thread newThread(Runnable r) {
    +					Thread t = Executors.defaultThreadFactory().newThread(r);
    +					t.setDaemon(true);
    +					t.setPriority(Thread.MIN_PRIORITY);
    +					return t;
    +				}
    +			});
    +
    +			// create the view in a separate thread (which will take some time
    +			// to complete)
    +			Future<Exception> future = executorService.submit(new CreateViewRunnable(fullTableName, fullViewName1));
    +			// wait till the thread makes the rpc to create the view
    +			latch1.await();
    +			tableDdl = "DROP TABLE " + fullTableName;
    +			try {
    +				// drop table should fail as we are concurrently adding a view
    +				conn.createStatement().execute(tableDdl);
    +				fail("Creating a view while concurrently dropping the base table should fail");
    +			} catch (ConcurrentTableMutationException e) {
    +			}
    +			latch2.countDown();
    +
    +			Exception e = future.get();
    +			assertTrue(e == null);
    +
    +			// create another view to ensure that the cell used to prevent
    +			// concurrent modifications was removed
    +			String ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
    +					+ " WHERE k = 6";
    +			conn.createStatement().execute(ddl);
    +		}
    +	}
    +
    +	@Test
    +	public void testConcurrentAddColumn() throws Exception {
    +		try (Connection conn = DriverManager.getConnection(getUrl())) {
    +			String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +			String fullViewName = SchemaUtil.getTableName(SCHEMA2, SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
    +			// create base table
    +			String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
    +					+ tableDDLOptions;
    +			conn.createStatement().execute(tableDdl);
    +			// create a view
    +			String ddl = "CREATE VIEW " + fullViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
    +					+ " WHERE k = 6";
    +			conn.createStatement().execute(ddl);
    +
    +			latch1 = new CountDownLatch(1);
    +			latch2 = new CountDownLatch(1);
    +			ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
    +				@Override
    +				public Thread newThread(Runnable r) {
    +					Thread t = Executors.defaultThreadFactory().newThread(r);
    +					t.setDaemon(true);
    +					t.setPriority(Thread.MIN_PRIORITY);
    +					return t;
    +				}
    +			});
    +
    +			// add a column to the view in a separate thread (which will take
    +			// some time to complete)
    +			Future<Exception> future = executorService.submit(new AddColumnRunnable(fullViewName));
    +			// wait till the thread makes the rpc to create the view
    +			boolean result = latch1.await(2, TimeUnit.MINUTES);
    +			if (!result) {
    +				fail("The create view rpc look too long");
    +			}
    +			tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
    +			try {
    +				// add the same column to the base table with a different type
    +				conn.createStatement().execute(tableDdl);
    +				fail("Creating a view while concurrently dropping the base table should fail");
    +			} catch (ConcurrentTableMutationException e) {
    +			}
    +			latch2.countDown();
    +
    +			Exception e = future.get();
    +			assertTrue(e == null);
    --- End diff --
    
    nit: `assertNull`


---

[GitHub] phoenix issue #313: PHOENIX-4799 Write cells using checkAndMutate to prevent...

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

    https://github.com/apache/phoenix/pull/313
  
    Just curious, why not use `SYSTEM.MUTEX` table for this? Its a table specially added for use-cases that need time based expiration and distributed locks.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r205950104
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3034,6 +3088,11 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                 return new MutationState(0, 0, connection);
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            // acquire a mutex on the table to prevent creating views while concurrently
    +            // dropping the base table
    +            if (tableType == PTableType.TABLE) {
    +                deleteCell(null, schemaName, tableName, null);
    +            }
    --- End diff --
    
    Done.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206737818
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -355,5 +355,19 @@
     			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
     			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
    +	
    +	 public static final String CREATE_MUTEX_METADTA =
    +	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +	            // Pk columns
    +	            TENANT_ID + " VARCHAR NULL," +
    +	            TABLE_SCHEM + " VARCHAR NULL," +
    +	            TABLE_NAME + " VARCHAR NOT NULL," +
    +	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	            HConstants.VERSIONS + "=%s,\n" +
    +	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
    +	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    --- End diff --
    
    
    Are you suggesting writeMutexCell/deleteMutexCell should take as arguments (tenantid, schema, tablename, column name, column family) instead of a byte[]? I will make that change.


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206416912
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -16,8 +16,29 @@
      */
     package org.apache.phoenix.end2end;
     
    -import com.google.common.base.Joiner;
    -import com.google.common.base.Throwables;
    +import static org.junit.Assert.assertEquals;
    --- End diff --
    
    nit: Diff generated due to change in order of imports


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r204969054
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -3034,6 +3086,10 @@ MutationState dropTable(String schemaName, String tableName, String parentTableN
                 return new MutationState(0, 0, connection);
             } finally {
                 connection.setAutoCommit(wasAutoCommit);
    +            // lock the table to prevent concurrent table modifications
    --- End diff --
    
    Update comment


---

[GitHub] phoenix pull request #313: PHOENIX-4799 Write cells using checkAndMutate to ...

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

    https://github.com/apache/phoenix/pull/313#discussion_r206688033
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -82,6 +82,10 @@
         static final String SYSTEM_SEQUENCE_IDENTIFIER =
                 QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
     
    +    static final String SYSTEM_MUTEX_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\""
    +                    + PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME + "\"";
    +
         static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    --- End diff --
    
    Done


---