Repository: apex-malhar Updated Branches: refs/heads/master 8e3a0c0b0 -> 42b4b1454
APEXMALHAR-2327 #resolve #comment BucketsFileSystem.writeBucketData() call Slice.toByteArray() cause allocate unnecessary memory Project: http://git-wip-us.apache.org/repos/asf/apex-malhar/repo Commit: http://git-wip-us.apache.org/repos/asf/apex-malhar/commit/42b4b145 Tree: http://git-wip-us.apache.org/repos/asf/apex-malhar/tree/42b4b145 Diff: http://git-wip-us.apache.org/repos/asf/apex-malhar/diff/42b4b145 Branch: refs/heads/master Commit: 42b4b1454c5b30feedcfde5ab2eadff83efc06cd Parents: 8e3a0c0 Author: brightchen <[email protected]> Authored: Thu Nov 3 15:38:16 2016 -0700 Committer: brightchen <[email protected]> Committed: Mon Nov 7 14:04:47 2016 -0800 ---------------------------------------------------------------------- .../java/com/datatorrent/lib/fileaccess/FileAccess.java | 10 ++++++++++ .../java/com/datatorrent/lib/fileaccess/TFileWriter.java | 8 ++++++++ .../apex/malhar/lib/state/managed/BucketsFileSystem.java | 4 ++-- 3 files changed, 20 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/42b4b145/library/src/main/java/com/datatorrent/lib/fileaccess/FileAccess.java ---------------------------------------------------------------------- diff --git a/library/src/main/java/com/datatorrent/lib/fileaccess/FileAccess.java b/library/src/main/java/com/datatorrent/lib/fileaccess/FileAccess.java index ebe1ef6..12f9036 100644 --- a/library/src/main/java/com/datatorrent/lib/fileaccess/FileAccess.java +++ b/library/src/main/java/com/datatorrent/lib/fileaccess/FileAccess.java @@ -147,13 +147,23 @@ public interface FileAccess extends Closeable { /** * Appends key/value pair to the underlying file. + * @deprecated use {@link FileWriter#append(Slice, Slice)} instead. * @param key * @param value * @throws IOException */ + @Deprecated void append(byte[] key, byte[] value) throws IOException; /** + * Appends key/value pair to the underlying file. + * @param key + * @param value + * @throws IOException + */ + void append(Slice key, Slice value) throws IOException; + + /** * Returns number of bytes written to the underlying stream. * @return The bytes written. * @throws IOException http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/42b4b145/library/src/main/java/com/datatorrent/lib/fileaccess/TFileWriter.java ---------------------------------------------------------------------- diff --git a/library/src/main/java/com/datatorrent/lib/fileaccess/TFileWriter.java b/library/src/main/java/com/datatorrent/lib/fileaccess/TFileWriter.java index da724d4..fed8447 100644 --- a/library/src/main/java/com/datatorrent/lib/fileaccess/TFileWriter.java +++ b/library/src/main/java/com/datatorrent/lib/fileaccess/TFileWriter.java @@ -25,6 +25,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.io.file.tfile.TFile.Writer; +import com.datatorrent.netlet.util.Slice; + /** * TFileWriter * @@ -59,6 +61,12 @@ public final class TFileWriter implements FileAccess.FileWriter } @Override + public void append(Slice key, Slice value) throws IOException + { + writer.append(key.buffer, key.offset, key.length, value.buffer, value.offset, value.length); + } + + @Override public long getBytesWritten() throws IOException { return fsdos.getPos(); http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/42b4b145/library/src/main/java/org/apache/apex/malhar/lib/state/managed/BucketsFileSystem.java ---------------------------------------------------------------------- diff --git a/library/src/main/java/org/apache/apex/malhar/lib/state/managed/BucketsFileSystem.java b/library/src/main/java/org/apache/apex/malhar/lib/state/managed/BucketsFileSystem.java index f65c539..2bd6ef7 100644 --- a/library/src/main/java/org/apache/apex/malhar/lib/state/managed/BucketsFileSystem.java +++ b/library/src/main/java/org/apache/apex/malhar/lib/state/managed/BucketsFileSystem.java @@ -173,7 +173,7 @@ public class BucketsFileSystem implements ManagedStateComponent dataSize += key.length; dataSize += value.length; - fileWriter.append(key.toByteArray(), value.toByteArray()); + fileWriter.append(key, value); if (firstKey == null) { firstKey = key; } @@ -197,7 +197,7 @@ public class BucketsFileSystem implements ManagedStateComponent dataSize += key.length; dataSize += value.length; - fileWriter.append(key.toByteArray(), value.toByteArray()); + fileWriter.append(key, value); if (firstKey == null) { firstKey = key; }
