You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by lu...@apache.org on 2018/05/26 02:27:28 UTC

asterixdb git commit: [NO ISSUE][STO] Improve error handling of writing deleted file

Repository: asterixdb
Updated Branches:
  refs/heads/master c7eb0c51a -> af17baa7d


[NO ISSUE][STO] Improve error handling of writing deleted file

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- When the bulkloader is aborted, the file would be deleted from the
buffer cache but there could still be unflushed pages in the write
queue. Instead of throwing exceptions, simply ignore the case when
writting to non-existing files (as before).

Change-Id: I3f693ef178e0ab68c2a28b0edc6ffeaaeca1c520
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2651
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <im...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/af17baa7
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/af17baa7
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/af17baa7

Branch: refs/heads/master
Commit: af17baa7df86542f106fdcb04caf1449800d6806
Parents: c7eb0c5
Author: luochen01 <cl...@uci.edu>
Authored: Mon May 21 17:12:43 2018 -0700
Committer: Luo Chen <cl...@uci.edu>
Committed: Fri May 25 19:26:14 2018 -0700

----------------------------------------------------------------------
 .../storage/common/buffercache/BufferCache.java       | 14 ++++++++------
 .../storage/common/buffercache/FIFOLocalWriter.java   |  5 +++++
 .../storage/common/file/BufferedFileHandle.java       |  8 ++++++++
 3 files changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/af17baa7/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 55ed75e..f8facf5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -588,7 +588,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
             fInfo = fileInfoMap.get(fileId);
         }
         if (fInfo == null) {
-            throw new HyracksDataException("No such file mapped");
+            throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileId);
         }
         return fInfo;
     }
@@ -607,11 +607,11 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
 
     void write(CachedPage cPage) throws HyracksDataException {
         BufferedFileHandle fInfo = getFileInfo(cPage);
-        if (fInfo == null) {
-            throw new IllegalStateException("Attempting to write non-existing file");
-        }
         // synchronize on fInfo to prevent the file handle from being deleted until the page is written.
         synchronized (fInfo) {
+            if (fInfo.fileHasBeenDeleted()) {
+                return;
+            }
             ByteBuffer buf = cPage.buffer.duplicate();
             final int totalPages = cPage.getFrameSizeMultiplier();
             final int extraBlockPageId = cPage.getExtraBlockPageId();
@@ -1032,13 +1032,15 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
                 }
             } finally {
                 try {
-                    ioManager.close(fInfo.getFileHandle());
+                    synchronized (fInfo) {
+                        ioManager.close(fInfo.getFileHandle());
+                        fInfo.markAsDeleted();
+                    }
                 } finally {
                     IoUtil.delete(fileRef);
                 }
             }
         }
-
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/af17baa7/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
index 9d0b728..856edbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
@@ -15,6 +15,7 @@
 
 package org.apache.hyracks.storage.common.buffercache;
 
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class FIFOLocalWriter implements IFIFOPageWriter {
@@ -29,6 +30,10 @@ public class FIFOLocalWriter implements IFIFOPageWriter {
         CachedPage cPage = (CachedPage) page;
         try {
             bufferCache.write(cPage);
+        } catch (HyracksDataException e) {
+            if (e.getErrorCode() != ErrorCode.FILE_DOES_NOT_EXIST) {
+                throw HyracksDataException.create(e);
+            }
         } finally {
             bufferCache.returnPage(cPage);
             if (DEBUG) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/af17baa7/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
index 62e7888..1312d97 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
@@ -45,6 +45,14 @@ public class BufferedFileHandle {
         return handle;
     }
 
+    public void markAsDeleted() {
+        handle = null;
+    }
+
+    public boolean fileHasBeenDeleted() {
+        return handle == null;
+    }
+
     public int incReferenceCount() {
         return refCount.incrementAndGet();
     }