Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-14350 Procedure V2 Phase 2: Assignment Manager
  3. HBASE-16233

Procedure V2: Support acquire/release shared table lock concurrently

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Major
    • Resolution: Fixed
    • None
    • 2.0.0
    • proc-v2
    • None

    Description

      MasterProcedureScheduler.TableQueue class only has one single instance of TableLock (private TableLock tableLock = null;) to track exclusive/shared table lock from TableLockManager.

      When multiple shared lock request comes, the later shared lock request would overwrite the lock acquired from earlier shared lock request, and hence, we will get some weird error when the second or later release lock request comes, because we lose track of the lock.

      The issue can be reproduced in the unit test of HBASE-14552. mbertozzi also comes up with a UT without using any real procedure to repro the problem:

      @Test
        public void testSchedWithZkLock() throws Exception {
          MiniZooKeeperCluster zkCluster = new MiniZooKeeperCluster(conf);
          int zkPort = zkCluster.startup(new File("/tmp/test-zk"));
          Thread.sleep(10000);
          conf.set("hbase.zookeeper.quorum", "localhost:" + zkPort);
      
          ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSchedWithZkLock", null, false);
          queue = new MasterProcedureScheduler(conf,
            TableLockManager.createTableLockManager(
              conf, zkw, ServerName.valueOf("localhost", 12345, 1)));
      
          final TableName tableName = TableName.valueOf("testtb");
          TestTableProcedure procA = new TestTableProcedure(1, tableName,
                TableProcedureInterface.TableOperationType.READ);
          TestTableProcedure procB = new TestTableProcedure(2, tableName,
                TableProcedureInterface.TableOperationType.READ);
      
          assertTrue(queue.tryAcquireTableSharedLock(procA, tableName));
          assertTrue(queue.tryAcquireTableSharedLock(procB, tableName));
      
          queue.releaseTableSharedLock(procA, tableName);
          queue.releaseTableSharedLock(procB, tableName);
        }
      

      Attachments

        1. HBASE-16233.v1-master.patch
          7 kB
          Stephen Yuan Jiang

        Activity

          People

            syuanjiang Stephen Yuan Jiang
            syuanjiang Stephen Yuan Jiang
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: