You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Anoop Sam John (JIRA)" <ji...@apache.org> on 2014/05/30 16:05:01 UTC

[jira] [Created] (HBASE-11275) [AccessController] postCreateTable hook fails when another CP creates table on their startup

Anoop Sam John created HBASE-11275:
--------------------------------------

             Summary: [AccessController] postCreateTable hook fails when another CP creates table on their startup
                 Key: HBASE-11275
                 URL: https://issues.apache.org/jira/browse/HBASE-11275
             Project: HBase
          Issue Type: Bug
          Components: security
            Reporter: Anoop Sam John
            Assignee: Anoop Sam John
             Fix For: 0.99.0, 0.98.4


I am using AC and VC together.( In the required order with AC first)
Still I am getting the below Exception
{code}
2014-05-29 14:10:55,574 ERROR [ActiveMasterManager] client.AsyncProcess(313): Failed to get region location 
org.apache.hadoop.hbase.TableNotFoundException: hbase:acl
       at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegionInMeta(ConnectionManager.java:1056)
       at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:995)
       at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:952)
       at org.apache.hadoop.hbase.client.AsyncProcess.findDestLocation(AsyncProcess.java:380)
       at org.apache.hadoop.hbase.client.AsyncProcess.submit(AsyncProcess.java:309)
       at org.apache.hadoop.hbase.client.AsyncProcess.submit(AsyncProcess.java:263)
       at org.apache.hadoop.hbase.client.HTable.backgroundFlushCommits(HTable.java:1022)
       at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1357)
       at org.apache.hadoop.hbase.client.HTable.put(HTable.java:957)
       at org.apache.hadoop.hbase.security.access.AccessControlLists.addUserPermission(AccessControlLists.java:174)
       at org.apache.hadoop.hbase.security.access.AccessController.postCreateTable(AccessController.java:867)
       at org.apache.hadoop.hbase.master.MasterCoprocessorHost.postCreateTable(MasterCoprocessorHost.java:258)
       at org.apache.hadoop.hbase.master.HMaster.createTable(HMaster.java:1123)
       at org.apache.hadoop.hbase.security.visibility.VisibilityController.postStartMaster(VisibilityController.java:231)
       at org.apache.hadoop.hbase.master.MasterCoprocessorHost.postStartMaster(MasterCoprocessorHost.java:1390)
       at org.apache.hadoop.hbase.master.HMaster.finishActiveMasterInitialization(HMaster.java:619)
       at org.apache.hadoop.hbase.master.HMaster.access$3(HMaster.java:462)
       at org.apache.hadoop.hbase.master.HMaster$1.run(HMaster.java:1258)
       at java.lang.Thread.run(Thread.java:745)
2014-05-29 14:10:55,587 ERROR [ActiveMasterManager] master.HMaster(621): Coprocessor postStartMaster() hook failed
org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed 1 action: hbase:acl: 1 time, 
       at org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.makeException(AsyncProcess.java:180)
       at org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.access$1(AsyncProcess.java:179)
       at org.apache.hadoop.hbase.client.AsyncProcess.waitForAllPreviousOpsAndReset(AsyncProcess.java:1024)
       at org.apache.hadoop.hbase.client.HTable.backgroundFlushCommits(HTable.java:1025)
       at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1357)
       at org.apache.hadoop.hbase.client.HTable.put(HTable.java:957)
       at org.apache.hadoop.hbase.security.access.AccessControlLists.addUserPermission(AccessControlLists.java:174)
       at org.apache.hadoop.hbase.security.access.AccessController.postCreateTable(AccessController.java:867)
       at org.apache.hadoop.hbase.master.MasterCoprocessorHost.postCreateTable(MasterCoprocessorHost.java:258)
       at org.apache.hadoop.hbase.master.HMaster.createTable(HMaster.java:1123)
       at org.apache.hadoop.hbase.security.visibility.VisibilityController.postStartMaster(VisibilityController.java:231)
       at org.apache.hadoop.hbase.master.MasterCoprocessorHost.postStartMaster(MasterCoprocessorHost.java:1390)
       at org.apache.hadoop.hbase.master.HMaster.finishActiveMasterInitialization(HMaster.java:619)
       at org.apache.hadoop.hbase.master.HMaster.access$3(HMaster.java:462)
       at org.apache.hadoop.hbase.master.HMaster$1.run(HMaster.java:1258)
       at java.lang.Thread.run(Thread.java:745)
{code}
During the master start, the postStartMaster on the AC is called 1st and it creates acl table.   This create will be handled by another thread in master.
Then CP f/w calls postStartMaster on VC which will try creating labels table.  
And in the AC we have a postCreateTable() hook where we add the table owner permission to acl table.  This hook will be called before the actual creation of the table (labels) and will be quickly called.
So by this time, when this hook on AC is getting executed, the table creation threads might be in process of creating the tables. (acl and labels tables)
And so we will get a TableNotFoundException.

Well this can be generic issue. Another CPs also might create tables. So I believe we should handle this in AC.

Solution will be simple
Handling of the owner permission to be moved to postCreateTableHandler() CP. This hook will be called after the table is fully created by the handler thread. I think we should add the table related permission only after it is fully created. 

Initailly I was thinking that we should add some waiting in the AC#postCreateTableHandler() so that the ACL table is created. (The table creations happen in another thread)
But seems this is not needed.
Even if the create table is given as a Threadpool.submit request, this pool  is having only one Thread.  So creation of the tables are happening in sequential way. 
Here is the master code with one thread
   {code}
   // We depend on there being only one instance of this executor running
   // at a time.  To do concurrency, would need fencing of enable/disable of
   // tables.
   this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
   {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)