Uploaded image for project: 'IMPALA'
  1. IMPALA
  2. IMPALA-4449

Revisit locking scheme in CatalogOpEx.alterTable()

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • Impala 2.5.0, Impala 2.6.0, Impala 2.7.0, Impala 2.8.0
    • Impala 2.9.0
    • Catalog

    Description

      IMPALA-1480 added a synchronized block on tbl that can potentially block other DDL operations.

       private void alterTable(TAlterTableParams params, TDdlExecResponse response)
            throws ImpalaException {
          // When true, loads the file/block metadata.
          boolean reloadFileMetadata = false;
          // When true, loads the table schema and the column stats from the Hive Metastore.
          boolean reloadTableSchema = false;
      
          TableName tableName = TableName.fromThrift(params.getTable_name());
          Table tbl = getExistingTable(tableName.getDb(), tableName.getTbl());
          catalog_.getLock().writeLock().lock();
          synchronized (tbl) {   -------------------------------> LOCK
            if (params.getAlter_type() == TAlterTableType.RENAME_VIEW
                || params.getAlter_type() == TAlterTableType.RENAME_TABLE) {
              // RENAME is implemented as an ADD + DROP, so we need to execute it as we hold
              // the catalog lock.
              try {
      

      Long running alterTable calls potentially block getCatalogObjects() below.

      public TGetAllCatalogObjectsResponse getCatalogObjects(long fromVersion) {
          TGetAllCatalogObjectsResponse resp = new TGetAllCatalogObjectsResponse();
          resp.setObjects(new ArrayList<TCatalogObject>());
          resp.setMax_catalog_version(Catalog.INITIAL_CATALOG_VERSION);
          catalogLock_.readLock().lock(); -----------------------------------------------------------------> Already obtained this read lock.
          try {
            for (Db db: getDbs(PatternMatcher.MATCHER_MATCH_ALL)) {
              TCatalogObject catalogDb = new TCatalogObject(TCatalogObjectType.DATABASE,
                  db.getCatalogVersion());
              catalogDb.setDb(db.toThrift());
              resp.addToObjects(catalogDb);
      
              for (String tblName: db.getAllTableNames()) {
                TCatalogObject catalogTbl = new TCatalogObject(TCatalogObjectType.TABLE,
                    Catalog.INITIAL_CATALOG_VERSION);
      
                Table tbl = db.getTable(tblName);
                if (tbl == null) {
                  LOG.error("Table: " + tblName + " was expected to be in the catalog " +
                      "cache. Skipping table for this update.");
                  continue;
                }
      
                // Protect the table from concurrent modifications.
                synchronized(tbl) {----------------------------------------> BLOCKED
      

      However getCatalogObjects has already obtained a global read lock on the catalog above catalogLock_.readLock().lock(); and this blocks any other DDL that tries to obtain a writeLock(). As an example, a insert query (on a totally urelated table) hangs with the following stack.

      "Thread-42924" prio=10 tid=0x0000000009b9c000 nid=0x70a5 waiting on condition [0x00007fc918d81000]
         java.lang.Thread.State: WAITING (parking)
      	at sun.misc.Unsafe.park(Native Method)
      	- parking to wait for  <0x00000007016a6f80> (a java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
      	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197)
      	at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:945)
      	at com.cloudera.impala.service.CatalogOpExecutor.updateCatalog(CatalogOpExecutor.java:2773)
      	at com.cloudera.impala.service.JniCatalog.updateCatalog(JniCatalog.java:248)
      

      TODO: Revisit the above locking scheme and figure out if there is a way to reduce the scope of the lock to minimize contention.

      Attachments

        Activity

          People

            dtsirogiannis Dimitris Tsirogiannis
            bharathv Bharath Vissapragada
            Votes:
            2 Vote for this issue
            Watchers:
            16 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: