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 2941a842e2 [ASTERIXDB-3407][STO] Increment page counter for valid pages
2941a842e2 is described below

commit 2941a842e2b7758aa7f647477134c8ce1b9ea285
Author: Wail Alkowaileet <[email protected]>
AuthorDate: Tue May 28 13:35:52 2024 -0700

    [ASTERIXDB-3407][STO] Increment page counter for valid pages
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    When reading a valid page from the buffer cache,
    the page counter in CloudMegaPageReadContext should
    be incremented.
    
    Change-Id: I7cfa419d216cf875a4e403c8b90b5a0007bf9bfc
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/18323
    Integration-Tests: Jenkins <[email protected]>
    Tested-by: Jenkins <[email protected]>
    Reviewed-by: Murtadha Hubail <[email protected]>
---
 .../buffercache/read/CloudMegaPageReadContext.java | 30 ++++++++++++++--------
 .../compression/file/CompressedFileManager.java    |  4 +++
 2 files changed, 24 insertions(+), 10 deletions(-)

diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/cloud/buffercache/read/CloudMegaPageReadContext.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/cloud/buffercache/read/CloudMegaPageReadContext.java
index 0f0b0b9506..f1b2fd40be 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/cloud/buffercache/read/CloudMegaPageReadContext.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/cloud/buffercache/read/CloudMegaPageReadContext.java
@@ -68,7 +68,7 @@ final class CloudMegaPageReadContext implements 
IBufferCacheReadContext {
     @Override
     public void onPin(ICachedPage page) throws HyracksDataException {
         CloudCachedPage cachedPage = (CloudCachedPage) page;
-        if (gapStream != null && cachedPage.skipCloudStream()) {
+        if (cachedPage.skipCloudStream()) {
             /*
              * This page is requested but the buffer cache has a valid copy in 
memory. Also, the page itself was
              * requested to be read from the cloud. Since this page is valid, 
no buffer cache read() will be performed.
@@ -76,14 +76,8 @@ final class CloudMegaPageReadContext implements 
IBufferCacheReadContext {
              * up writing the bytes of this page in the position of another 
page. Therefore, we should skip the bytes
              * for this particular page to avoid placing the bytes of this 
page into another page's position.
              */
-            try {
-                long remaining = cachedPage.getCompressedPageSize();
-                while (remaining > 0) {
-                    remaining -= gapStream.skip(remaining);
-                }
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
+            skipCloudBytes(cachedPage);
+            pageCounter++;
         }
     }
 
@@ -187,16 +181,32 @@ final class CloudMegaPageReadContext implements 
IBufferCacheReadContext {
             return gapStream;
         }
 
-        LOGGER.info("Cloud stream read for {} pages", numberOfContiguousPages 
- pageCounter);
         int requiredNumOfPages = numberOfContiguousPages - pageCounter;
         long offset = cPage.getCompressedPageOffset();
         int pageId = BufferedFileHandle.getPageId(cPage.getDiskPageId());
         long length = fileHandle.getPagesTotalSize(pageId, requiredNumOfPages);
 
+        LOGGER.info("Cloud stream read for {} pages [{}, {}]", 
numberOfContiguousPages - pageCounter, pageId,
+                pageId + requiredNumOfPages);
         ICloudIOManager cloudIOManager = (ICloudIOManager) ioManager;
         gapStream = cloudIOManager.cloudRead(fileHandle.getFileHandle(), 
offset, length);
 
         return gapStream;
     }
 
+    private void skipCloudBytes(CloudCachedPage cachedPage) throws 
HyracksDataException {
+        if (gapStream == null) {
+            return;
+        }
+
+        try {
+            long remaining = cachedPage.getCompressedPageSize();
+            while (remaining > 0) {
+                remaining -= gapStream.skip(remaining);
+            }
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
index 365b92d296..da908b160e 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
@@ -258,6 +258,10 @@ public class CompressedFileManager {
     }
 
     public long getTotalCompressedSize(int startPageId, int numberOfPages) 
throws HyracksDataException {
+        if (startPageId + numberOfPages > totalNumOfPages) {
+            throw new IndexOutOfBoundsException(startPageId + " + " + 
numberOfPages + " > " + totalNumOfPages);
+        }
+
         int lafPageId = -1;
         ICachedPage lafPage = null;
         long totalSize = 0;

Reply via email to