This is an automated email from the ASF dual-hosted git repository.

wyk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 110f55950c [NO ISSUE][CLUS] Avoid locking resources while waiting for 
IO
110f55950c is described below

commit 110f55950cbb9c60dc4bc5a92dece869089fd87c
Author: Wail Alkowaileet <[email protected]>
AuthorDate: Mon Jun 24 20:54:11 2024 -0700

    [NO ISSUE][CLUS] Avoid locking resources while waiting for IO
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    Waiting for all I/O operations should not acquire a lock on
    when cleaning up a partition.
    
    Change-Id: I767474703e5519bf576db63fc988aeb67aebffe4
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/18400
    Integration-Tests: Jenkins <[email protected]>
    Tested-by: Jenkins <[email protected]>
    Reviewed-by: Murtadha Hubail <[email protected]>
---
 .../org/apache/asterix/common/context/DatasetInfo.java   |  4 ++++
 .../management/resource/CleanupBlockingIOOperation.java  |  2 +-
 .../resource/PersistentLocalResourceRepository.java      | 16 ++++++++--------
 3 files changed, 13 insertions(+), 9 deletions(-)

diff --git 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index 9064db5bbf..c8db4cd961 100644
--- 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -266,6 +266,8 @@ public class DatasetInfo extends Info implements 
Comparable<DatasetInfo> {
         synchronized (this) {
             while (partitionPendingIO.getOrDefault(partition, 0) > 0) {
                 try {
+                    int numPendingIOOps = 
partitionPendingIO.getOrDefault(partition, 0);
+                    LOGGER.debug("Waiting for {} IO operations in {} partition 
{}", numPendingIOOps, this, partition);
                     wait();
                 } catch (InterruptedException e) {
                     Thread.currentThread().interrupt();
@@ -273,6 +275,8 @@ public class DatasetInfo extends Info implements 
Comparable<DatasetInfo> {
                 }
             }
 
+            LOGGER.debug("All IO operations for {} partition {} are finished", 
this, partition);
+
             Set<IndexInfo> indexes = partitionIndexes.get(partition);
             if (indexes != null) {
                 // Perform the required operation
diff --git 
a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/CleanupBlockingIOOperation.java
 
b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/CleanupBlockingIOOperation.java
index 0790a22b52..6875ebd8b5 100644
--- 
a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/CleanupBlockingIOOperation.java
+++ 
b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/CleanupBlockingIOOperation.java
@@ -62,7 +62,7 @@ class CleanupBlockingIOOperation implements 
IIOBlockingOperation {
     }
 
     /**
-     * Clean all inactive indexes while the DatasetLifeCycleManager is 
synchronized
+     * Clean all inactive indexes
      */
     @Override
     public void afterOperation() throws HyracksDataException {
diff --git 
a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
 
b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index d0b4aa1beb..5c7d5ac366 100644
--- 
a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ 
b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -559,14 +559,10 @@ public class PersistentLocalResourceRepository implements 
ILocalResourceReposito
     }
 
     public void cleanup(int partition) throws HyracksDataException {
-        beforeReadAccess();
-        try {
-            CleanupBlockingIOOperation cleanupOp = new 
CleanupBlockingIOOperation(partition, this, ioManager);
-            
datasetLifecycleManager.waitForIOAndPerform(AllDatasetsReplicationStrategy.INSTANCE,
 partition, cleanupOp);
-        } finally {
-            clearResourcesCache();
-            afterReadAccess();
-        }
+        LOGGER.info("Cleaning up indexes in partition {}", partition);
+        CleanupBlockingIOOperation cleanupOp = new 
CleanupBlockingIOOperation(partition, this, ioManager);
+        
datasetLifecycleManager.waitForIOAndPerform(AllDatasetsReplicationStrategy.INSTANCE,
 partition, cleanupOp);
+        LOGGER.info("Finished cleaning up indexes in partition {} 
successfully", partition);
     }
 
     /**
@@ -575,6 +571,7 @@ public class PersistentLocalResourceRepository implements 
ILocalResourceReposito
      * @param index to clean
      */
     void cleanupIndex(FileReference index) throws HyracksDataException {
+        beforeReadAccess();
         try {
             deleteIndexMaskedFiles(index);
             if (isValidIndex(index)) {
@@ -582,6 +579,9 @@ public class PersistentLocalResourceRepository implements 
ILocalResourceReposito
             }
         } catch (IOException | ParseException e) {
             throw HyracksDataException.create(e);
+        } finally {
+            clearResourcesCache();
+            afterReadAccess();
         }
     }
 

Reply via email to