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

alsuliman 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 ab2c5a2f60 [NO ISSUE][STO] Add logs when not able to enter a component
ab2c5a2f60 is described below

commit ab2c5a2f6033255ab42759bd68b5e9d4083997dc
Author: Ali Alsuliman <[email protected]>
AuthorDate: Tue May 21 23:08:25 2024 +0300

    [NO ISSUE][STO] Add logs when not able to enter a component
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Change-Id: Ieb6798e8e4df4b725009b71785ab78cfca214786
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/18303
    Integration-Tests: Jenkins <[email protected]>
    Tested-by: Jenkins <[email protected]>
    Reviewed-by: Murtadha Hubail <[email protected]>
---
 .../common/context/GlobalVirtualBufferCache.java   | 19 ++++++++++++++-----
 .../storage/am/lsm/common/api/ILSMComponent.java   |  4 ++++
 .../am/lsm/common/api/IVirtualBufferCache.java     |  4 ++++
 .../common/impls/AbstractLSMMemoryComponent.java   |  5 +++++
 .../storage/am/lsm/common/impls/LSMHarness.java    | 22 +++++++++++++++++-----
 5 files changed, 44 insertions(+), 10 deletions(-)

diff --git 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
index 1ba9ac6273..3eb1e4cdd4 100644
--- 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
+++ 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
@@ -189,7 +189,7 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
                     opTracker.notifyAll();
                 }
             }
-            checkAndNotifyFlushThread();
+            checkAndNotifyFlushThread(false);
         }
         if (memoryComponent.getLsmIndex().getNumOfFilterFields() > 0
                 && memoryComponent.getLsmIndex().isPrimaryIndex()) {
@@ -210,7 +210,7 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
     public boolean isFull() {
         boolean full = vbc.isFull();
         if (full) {
-            checkAndNotifyFlushThread();
+            checkAndNotifyFlushThread(true);
         }
         return full;
     }
@@ -280,7 +280,7 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
         ICachedPage page = vbc.pin(dpid, context);
         if (context.isNewPage()) {
             incrementFilteredMemoryComponentUsage(dpid, 1);
-            checkAndNotifyFlushThread();
+            checkAndNotifyFlushThread(false);
         }
         return page;
     }
@@ -299,8 +299,12 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
         }
     }
 
-    private void checkAndNotifyFlushThread() {
+    private void checkAndNotifyFlushThread(boolean log) {
         if (vbc.getUsage() < flushPageBudget) {
+            if (log) {
+                LOGGER.info("not notifying the flush thread vbcUsage({}) < 
flushPageBudget({})", vbc.getUsage(),
+                        flushPageBudget);
+            }
             return;
         }
         // Notify the flush thread to schedule flushes. This is used to avoid 
deadlocks because page pins can be
@@ -317,7 +321,7 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
         int delta = multiplier - cPage.getFrameSizeMultiplier();
         incrementFilteredMemoryComponentUsage(((VirtualPage) cPage).dpid(), 
delta);
         if (delta > 0) {
-            checkAndNotifyFlushThread();
+            checkAndNotifyFlushThread(false);
         }
     }
 
@@ -449,6 +453,11 @@ public class GlobalVirtualBufferCache implements 
IVirtualBufferCache, ILifeCycle
         return vbc.toString();
     }
 
+    @Override
+    public String dumpState() {
+        return "flushingComponents=" + flushingComponents + ", " + 
vbc.dumpState();
+    }
+
     @Override
     public void closeFileIfOpen(FileReference fileRef) {
         vbc.closeFileIfOpen(fileRef);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index 186cabbc4e..7de62ce36a 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -155,4 +155,8 @@ public interface ILSMComponent {
      * @return the number of readers inside a component
      */
     int getReaderCount();
+
+    default String dumpState() {
+        return "";
+    }
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
index bbe6051c41..16c6c1096d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
@@ -63,4 +63,8 @@ public interface IVirtualBufferCache extends IBufferCache {
      * @throws HyracksDataException
      */
     void flushed(ILSMMemoryComponent memoryComponent) throws 
HyracksDataException;
+
+    default String dumpState() {
+        return "";
+    }
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 7088791aa7..a42f64b338 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -355,4 +355,9 @@ public abstract class AbstractLSMMemoryComponent extends 
AbstractLSMComponent im
                 + writerCount + ", \"readers\":" + readerCount + ", 
\"pendingFlushes\":" + pendingFlushes
                 + ", \"id\":\"" + componentId + "\", \"index\":" + getIndex() 
+ "}";
     }
+
+    @Override
+    public String dumpState() {
+        return this + ", isCompFull=" + vbc.isFull(this) + ", isFull=" + 
vbc.isFull() + ", " + vbc.dumpState();
+    }
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index cdf7ad7a17..28567c00dd 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -22,6 +22,7 @@ package org.apache.hyracks.storage.am.lsm.common.impls;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Predicate;
 
@@ -100,14 +101,22 @@ public class LSMHarness implements ILSMHarness {
         }
         try {
             validateOperationEnterComponentsState(ctx);
+            long start = System.nanoTime();
+            boolean dumpState = false;
             synchronized (opTracker) {
                 while (true) {
                     lsmIndex.getOperationalComponents(ctx);
-                    if (enterComponents(ctx, opType)) {
+                    if (enterComponents(ctx, opType, dumpState)) {
                         return true;
                     } else if (isTryOperation) {
                         return false;
                     }
+                    dumpState = false;
+                    long minutes = 
TimeUnit.NANOSECONDS.toMinutes(System.nanoTime() - start);
+                    if (minutes > 2) {
+                        start = System.nanoTime();
+                        dumpState = true;
+                    }
                     try {
                         opTracker.wait(100);
                     } catch (InterruptedException e) {
@@ -124,7 +133,7 @@ public class LSMHarness implements ILSMHarness {
     }
 
     @CriticalPath
-    protected boolean enterComponents(ILSMIndexOperationContext ctx, 
LSMOperationType opType)
+    protected boolean enterComponents(ILSMIndexOperationContext ctx, 
LSMOperationType opType, boolean dumpState)
             throws HyracksDataException {
         validateOperationEnterComponentsState(ctx);
         List<ILSMComponent> components = ctx.getComponentHolder();
@@ -136,6 +145,9 @@ public class LSMHarness implements ILSMHarness {
                 final ILSMComponent component = components.get(i);
                 boolean isMutableComponent = numEntered == 0 && 
component.getType() == LSMComponentType.MEMORY;
                 if (!component.threadEnter(opType, isMutableComponent)) {
+                    if (dumpState) {
+                        LOGGER.info("couldn't enter component: {}", 
component.dumpState());
+                    }
                     break;
                 }
                 numEntered++;
@@ -523,7 +535,7 @@ public class LSMHarness implements ILSMHarness {
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
         LOGGER.debug("Started a flush operation for index: {}", lsmIndex);
         synchronized (opTracker) {
-            while (!enterComponents(operation.getAccessor().getOpContext(), 
LSMOperationType.FLUSH)) {
+            while (!enterComponents(operation.getAccessor().getOpContext(), 
LSMOperationType.FLUSH, false)) {
                 try {
                     opTracker.wait();
                 } catch (InterruptedException e) {
@@ -589,7 +601,7 @@ public class LSMHarness implements ILSMHarness {
                     mergeOp.getMergingComponents().size(), lsmIndex);
         }
         synchronized (opTracker) {
-            enterComponents(operation.getAccessor().getOpContext(), 
LSMOperationType.MERGE);
+            enterComponents(operation.getAccessor().getOpContext(), 
LSMOperationType.MERGE, false);
         }
         ILSMDiskComponent newComponent;
         try {
@@ -914,7 +926,7 @@ public class LSMHarness implements ILSMHarness {
                 exitComponents(componentReplacementCtx, 
LSMOperationType.SEARCH, null, false);
                 // enter new component
                 componentReplacementCtx.prepareToEnter();
-                enterComponents(componentReplacementCtx, 
LSMOperationType.SEARCH);
+                enterComponents(componentReplacementCtx, 
LSMOperationType.SEARCH, false);
                 componentReplacementCtx.replace(ctx);
             }
         }

Reply via email to