You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "huyuanfeng2018 (via GitHub)" <gi...@apache.org> on 2023/02/13 14:59:09 UTC

[GitHub] [iceberg] huyuanfeng2018 opened a new issue, #6824: iceberg can not write data when last unlock error use hive catalog

huyuanfeng2018 opened a new issue, #6824:
URL: https://github.com/apache/iceberg/issues/6824

   ### Apache Iceberg version
   
   None
   
   ### Query engine
   
   None
   
   ### Please describe the bug 🐞
   
   When I use the hive catalog to manage the iceberg table, every time I take a snapshot, I will apply for the lock of the corresponding hive table and unlock it after the writing is completed, but when the unlock fails, no exception will be thrown, which will lead to the following A snapshot cycle can never acquire a lock, the task cannot continue to write, and the small file merge program cannot be called at the same time.
   Corresponding code to unlock
   :
   ```
     private void unlock(Optional<Long> lockId) {
       if (lockId.isPresent()) {
         try {
           doUnlock(lockId.get());
         } catch (Exception e) {
           LOG.warn("Failed to unlock {}.{}", database, tableName, e);
         }
       }
     }
   ```
   get lock :
   ```
                 try {
                   LockResponse response = metaClients.run(client -> client.checkLock(id));
                   LockState newState = response.getState();
                   state.set(newState);
                   if (newState.equals(LockState.WAITING)) {
                     throw new WaitingForLockException("Waiting for lock.");
                   }
                 } catch (InterruptedException e) {
                   Thread.interrupted(); // Clear the interrupt status flag
                   LOG.warn("Interrupted while waiting for lock.", e);
                 }
               }, TException.class);
   ```
   
   It seems that there is always this problem


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] huyuanfeng2018 commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "huyuanfeng2018 (via GitHub)" <gi...@apache.org>.
huyuanfeng2018 commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1428089327

   So we can see how to solve this problem. 
   my opinion is
   Is it possible to store locked in the state after the unlocking fails. At the beginning of the next ckp cycle, if there is a state where the unlocking failed, try to unlock it first?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] huyuanfeng2018 commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "huyuanfeng2018 (via GitHub)" <gi...@apache.org>.
huyuanfeng2018 commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1612968023

   > > #6451 Try to helps in most of the case #6570 might be interesting for you as well
   > 
   > Hi,This doesn't seem to solve the problem
   
   I think so too
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] huyuanfeng2018 commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "huyuanfeng2018 (via GitHub)" <gi...@apache.org>.
huyuanfeng2018 commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1612967214

   > 
   
   I think so too
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lurnagao commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "lurnagao (via GitHub)" <gi...@apache.org>.
lurnagao commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1451430014

   > #6451 Try to helps in most of the case #6570 might be interesting for you as well
   
   Hi,This doesn't seem to solve the problem


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1438813798

   #6451 Try to helps in most of the case
   #6570 might be interesting for you as well


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] iceberg can not write data when last unlock error use hive catalog [iceberg]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1880238758

   This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To permanently prevent this issue from being considered stale, add the label 'not-stale', but commenting on the issue is preferred when possible.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] huyuanfeng2018 commented on issue #6824: iceberg can not write data when last unlock error use hive catalog

Posted by "huyuanfeng2018 (via GitHub)" <gi...@apache.org>.
huyuanfeng2018 commented on issue #6824:
URL: https://github.com/apache/iceberg/issues/6824#issuecomment-1428094899

   @raptond 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org