Luo Chen has uploaded a new change for review.
https://asterix-gerrit.ics.uci.edu/2604
Change subject: [NO ISSUE][STO] Avoid memory leak after delete file from
BufferCache
......................................................................
[NO ISSUE][STO] Avoid memory leak after delete file from BufferCache
- user model changes: no
- storage format changes: no
- interface changes: no
Details:
- Currently whwn we delete a file from a buffer cache, we didn't
remove the file info immediately after the FileInfoMap. However,
typically we often set max open files as a max integer number.
After the system is running, it's possible that the FileInfoMap could
contain a huge number of entries. This patch deletes the file info
immediately when the caller wants to delete a file.
Change-Id: I8a199d3e83592425ab5055bd12bde519e80deb13
---
M
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
M
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
2 files changed, 31 insertions(+), 48 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb
refs/changes/04/2604/1
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 1443bbc..63acb54 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
@@ -77,7 +77,7 @@
private final Queue<BufferCacheHeaderHelper> headerPageCache = new
ConcurrentLinkedQueue<>();
//DEBUG
- private Level fileOpsLevel = Level.DEBUG;
+ private final Level fileOpsLevel = Level.DEBUG;
private ArrayList<CachedPage> confiscatedPages;
private Lock confiscateLock;
private HashMap<CachedPage, StackTraceElement[]> confiscatedPagesOwner;
@@ -607,34 +607,34 @@
void write(CachedPage cPage) throws HyracksDataException {
BufferedFileHandle fInfo = getFileInfo(cPage);
+ if (fInfo == null) {
+ throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST);
+ }
// synchronize on fInfo to prevent the file handle from being deleted
until the page is written.
synchronized (fInfo) {
- if (!fInfo.fileHasBeenDeleted()) {
- ByteBuffer buf = cPage.buffer.duplicate();
- final int totalPages = cPage.getFrameSizeMultiplier();
- final int extraBlockPageId = cPage.getExtraBlockPageId();
- final boolean contiguousLargePages =
(BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
- BufferCacheHeaderHelper header = checkoutHeaderHelper();
- try {
- buf.limit(contiguousLargePages ? pageSize * totalPages :
pageSize);
- buf.position(0);
- long bytesWritten =
ioManager.syncWrite(fInfo.getFileHandle(),
-
getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)),
- header.prepareWrite(cPage, buf));
+ ByteBuffer buf = cPage.buffer.duplicate();
+ final int totalPages = cPage.getFrameSizeMultiplier();
+ final int extraBlockPageId = cPage.getExtraBlockPageId();
+ final boolean contiguousLargePages =
(BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
+ BufferCacheHeaderHelper header = checkoutHeaderHelper();
+ try {
+ buf.limit(contiguousLargePages ? pageSize * totalPages :
pageSize);
+ buf.position(0);
+ long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
+
getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)),
header.prepareWrite(cPage, buf));
- if (bytesWritten != (contiguousLargePages ? pageSize *
(totalPages - 1) : 0)
- + getPageSizeWithHeader()) {
- throw new HyracksDataException("Failed to write
completely: " + bytesWritten);
- }
- } finally {
- returnHeaderHelper(header);
+ if (bytesWritten != (contiguousLargePages ? pageSize *
(totalPages - 1) : 0)
+ + getPageSizeWithHeader()) {
+ throw new HyracksDataException("Failed to write
completely: " + bytesWritten);
}
- if (totalPages > 1 && !contiguousLargePages) {
- buf.limit(totalPages * pageSize);
- ioManager.syncWrite(fInfo.getFileHandle(),
getOffsetForPage(extraBlockPageId), buf);
- }
- assert buf.capacity() == (pageSize * totalPages);
+ } finally {
+ returnHeaderHelper(header);
}
+ if (totalPages > 1 && !contiguousLargePages) {
+ buf.limit(totalPages * pageSize);
+ ioManager.syncWrite(fInfo.getFileHandle(),
getOffsetForPage(extraBlockPageId), buf);
+ }
+ assert buf.capacity() == (pageSize * totalPages);
}
}
@@ -786,11 +786,8 @@
synchronized (fileInfoMap) {
fileInfoMap.forEach((key, value) -> {
try {
- boolean fileHasBeenDeleted = value.fileHasBeenDeleted();
- sweepAndFlush(key, !fileHasBeenDeleted);
- if (!fileHasBeenDeleted) {
- ioManager.close(value.getFileHandle());
- }
+ sweepAndFlush(key, true);
+ ioManager.close(value.getFileHandle());
} catch (HyracksDataException e) {
if (LOGGER.isWarnEnabled()) {
LOGGER.log(Level.WARN, "Error flushing file id: " +
key, e);
@@ -855,11 +852,8 @@
for (Map.Entry<Integer, BufferedFileHandle> entry :
fileInfoMap.entrySet()) {
if (entry.getValue().getReferenceCount() <= 0) {
int entryFileId = entry.getKey();
- boolean fileHasBeenDeleted =
entry.getValue().fileHasBeenDeleted();
- sweepAndFlush(entryFileId, !fileHasBeenDeleted);
- if (!fileHasBeenDeleted) {
-
ioManager.close(entry.getValue().getFileHandle());
- }
+ sweepAndFlush(entryFileId, true);
+ ioManager.close(entry.getValue().getFileHandle());
fileInfoMap.remove(entryFileId);
unreferencedFileFound = true;
// for-each iterator is invalid because we changed
@@ -997,7 +991,7 @@
sweepAndFlush(fileId, false);
BufferedFileHandle fInfo = null;
try {
- fInfo = fileInfoMap.get(fileId);
+ fInfo = fileInfoMap.remove(fileId);
if (fInfo != null && fInfo.getReferenceCount() > 0) {
throw new HyracksDataException("Deleting open file");
}
@@ -1011,10 +1005,7 @@
// such that when its pages are reclaimed in
openFile(),
// the pages are not flushed to disk but only
invalidated.
synchronized (fInfo) {
- if (!fInfo.fileHasBeenDeleted()) {
- ioManager.close(fInfo.getFileHandle());
- fInfo.markAsDeleted();
- }
+ ioManager.close(fInfo.getFileHandle());
}
}
} finally {
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 177128e..85ddfdb 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
@@ -24,7 +24,7 @@
public class BufferedFileHandle {
private final int fileId;
- private IFileHandle handle;
+ private final IFileHandle handle;
private final AtomicInteger refCount;
public BufferedFileHandle(int fileId, IFileHandle handle) {
@@ -39,14 +39,6 @@
public IFileHandle getFileHandle() {
return handle;
- }
-
- public void markAsDeleted() {
- handle = null;
- }
-
- public boolean fileHasBeenDeleted() {
- return handle == null;
}
public int incReferenceCount() {
--
To view, visit https://asterix-gerrit.ics.uci.edu/2604
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newchange
Gerrit-Change-Id: I8a199d3e83592425ab5055bd12bde519e80deb13
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Luo Chen <[email protected]>