Repository: tajo Updated Branches: refs/heads/master 1867c53e4 -> 3df7fda36
TAJO-953: RawFile should release a DirectBuffer immediately. (jinho) Closes #79 Project: http://git-wip-us.apache.org/repos/asf/tajo/repo Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/3df7fda3 Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/3df7fda3 Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/3df7fda3 Branch: refs/heads/master Commit: 3df7fda36bfcb3869878f72d90830315b0584f2e Parents: 1867c53 Author: jinossy <[email protected]> Authored: Thu Jul 17 19:07:24 2014 +0900 Committer: jinossy <[email protected]> Committed: Thu Jul 17 19:07:24 2014 +0900 ---------------------------------------------------------------------- CHANGES | 2 ++ .../java/org/apache/tajo/storage/RawFile.java | 18 +++++++++++------- .../java/org/apache/tajo/storage/StorageUtil.java | 18 ++++++++++++++++-- 3 files changed, 29 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tajo/blob/3df7fda3/CHANGES ---------------------------------------------------------------------- diff --git a/CHANGES b/CHANGES index 86f40c9..a67ded8 100644 --- a/CHANGES +++ b/CHANGES @@ -29,6 +29,8 @@ Release 0.9.0 - unreleased IMPROVEMENT + TAJO-953: RawFile should release a DirectBuffer immediately. (jinho) + TAJO-933: Fork some classes of Parquet as builtin third-party classes. (hyunsik) http://git-wip-us.apache.org/repos/asf/tajo/blob/3df7fda3/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java ---------------------------------------------------------------------- diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java index 9677bca..41d1e05 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.statistics.TableStats; @@ -92,7 +93,7 @@ public class RawFile { LOG.debug("RawFileScanner open:" + path + "," + channel.position() + ", size :" + channel.size()); } - buffer = ByteBuffer.allocateDirect(128 * 1024); + buffer = ByteBuffer.allocateDirect(64 * 1024); columnTypes = new DataType[schema.size()]; for (int i = 0; i < schema.size(); i++) { @@ -378,9 +379,9 @@ public class RawFile { tableStats.setReadBytes(fileSize); tableStats.setNumRows(recordCount); } - buffer.clear(); - channel.close(); - fis.close(); + + StorageUtil.closeBuffer(buffer); + IOUtils.cleanup(LOG, channel, fis); } @Override @@ -706,7 +707,9 @@ public class RawFile { @Override public void flush() throws IOException { - flushBuffer(); + if(buffer != null){ + flushBuffer(); + } } @Override @@ -718,8 +721,9 @@ public class RawFile { if (LOG.isDebugEnabled()) { LOG.debug("RawFileAppender written: " + getOffset() + " bytes, path: " + path); } - channel.close(); - randomAccessFile.close(); + + StorageUtil.closeBuffer(buffer); + IOUtils.cleanup(LOG, channel, randomAccessFile); } @Override http://git-wip-us.apache.org/repos/asf/tajo/blob/3df7fda3/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java ---------------------------------------------------------------------- diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java index d11dc09..95bb96f 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java @@ -23,17 +23,21 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.tajo.catalog.*; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.util.FileUtil; import org.apache.tajo.util.KeyValueSet; import parquet.hadoop.ParquetOutputFormat; +import sun.nio.ch.DirectBuffer; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -public class StorageUtil extends StorageConstants{ +public class StorageUtil extends StorageConstants { public static int getRowByteSize(Schema schema) { int sum = 0; for(Column col : schema.getColumns()) { @@ -185,4 +189,14 @@ public class StorageUtil extends StorageConstants{ return -1; } } + + public static void closeBuffer(ByteBuffer buffer) { + if (buffer != null) { + if (buffer.isDirect()) { + ((DirectBuffer) buffer).cleaner().clean(); + } else { + buffer.clear(); + } + } + } }
