feature: optimize data receiving (#181)

* use zero-copy optimize data recv
* remove fail message
* improve some config
* support buffer file sort
* support sort with buffer file and fix some unit test
* create target file
* rebuild sort module

Co-authored-by: ShouJing <1075185785@qq.com>
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/AbstractPointerCombiner.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/AbstractPointerCombiner.java
index 62e8153..768736a 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/AbstractPointerCombiner.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/AbstractPointerCombiner.java
@@ -28,8 +28,8 @@
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.Readable;
 import com.baidu.hugegraph.computer.core.io.Writable;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InlinePointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.InlinePointer;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 
 public abstract class AbstractPointerCombiner<T extends Readable & Writable>
                 implements PointerCombiner {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombiner.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombiner.java
index 8932f82..6fdf9ea 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombiner.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombiner.java
@@ -1,6 +1,6 @@
 package com.baidu.hugegraph.computer.core.combiner;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 
 public interface PointerCombiner {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/VertexValueCombiner.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/VertexValueCombiner.java
index fb84b95..cde0c42 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/VertexValueCombiner.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/combiner/VertexValueCombiner.java
@@ -25,8 +25,8 @@
 import com.baidu.hugegraph.computer.core.graph.properties.Properties;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.StreamGraphInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InlinePointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.InlinePointer;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 
 public class VertexValueCombiner extends AbstractPointerCombiner<Properties> {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java
index 06c9e84..981fecc 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java
@@ -29,6 +29,7 @@
     private static final long serialVersionUID = -6538118382075299762L;
 
     public static final int DEFAULT_CODE = 0;
+    public static final int REMOTE_ERROR_CODE = 1;
 
     private final int errorCode;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManager.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManager.java
index a78a26f..6517fcb 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManager.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManager.java
@@ -37,9 +37,9 @@
 import com.baidu.hugegraph.computer.core.receiver.MessageStat;
 import com.baidu.hugegraph.computer.core.sender.MessageSendManager;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
 import com.baidu.hugegraph.computer.core.util.Consumers;
 import com.baidu.hugegraph.computer.core.worker.WorkerContext;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.computer.core.worker.WorkerStat;
 import com.baidu.hugegraph.util.ExecutorUtil;
 import com.baidu.hugegraph.util.Log;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/FileGraphPartition.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/FileGraphPartition.java
index 8f8bab6..38f3df7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/FileGraphPartition.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/FileGraphPartition.java
@@ -43,10 +43,10 @@
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.store.FileGenerator;
 import com.baidu.hugegraph.computer.core.store.FileManager;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.computer.core.worker.Computation;
 import com.baidu.hugegraph.computer.core.worker.ComputationContext;
 import com.baidu.hugegraph.computer.core.worker.WorkerContext;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInput.java
index a2d4640..5650154 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInput.java
@@ -32,8 +32,8 @@
 import com.baidu.hugegraph.computer.core.io.BytesInput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 
 public class MessageInput<T extends Value> {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/ReusablePointer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/ReusablePointer.java
index 9231759..a20139d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/ReusablePointer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/compute/input/ReusablePointer.java
@@ -27,7 +27,7 @@
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
 import com.baidu.hugegraph.computer.core.io.Readable;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.computer.core.util.BytesUtil;
 
 public class ReusablePointer implements Pointer, Readable {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
index e4e9fb7..6dc4ac4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
@@ -586,7 +586,7 @@
                     "trigger sorting, the write buffer is used to store " +
                     "vertex or message.",
                     positiveInt(),
-                    (int) (50 * Bytes.KB)
+                    (int) (50 * Bytes.MB)
             );
 
     public static final ConfigOption<Integer>
@@ -595,7 +595,7 @@
                     "The initial size of write buffer that used to store " +
                     "vertex or message.",
                     positiveInt(),
-                    (int) (60 * Bytes.KB)
+                    (int) (50 * Bytes.MB)
             );
 
     public static final ConfigOption<Class<?>> MASTER_COMPUTATION_CLASS =
@@ -682,9 +682,19 @@
                     false
             );
 
+    public static final ConfigOption<Boolean> TRANSPORT_RECV_FILE_MODE =
+            new ConfigOption<>(
+                    "transport.recv_file_mode",
+                    "Whether enable receive buffer-file mode, it will " +
+                    "receive buffer write file from socket by " +
+                    "zero-copy if enable.",
+                    allowValues(true, false),
+                    true
+            );
+
     public static final ConfigOption<Boolean> TRANSPORT_TCP_KEEP_ALIVE =
             new ConfigOption<>(
-                    "transport.transport_tcp_keep_alive",
+                    "transport.tcp_keep_alive",
                     "Whether enable TCP keep-alive.",
                     allowValues(true, false),
                     true
@@ -740,7 +750,7 @@
                     "The timeout(in ms) to wait response after " +
                     "sending sync-request.",
                     positiveInt(),
-                    5_000L
+                    10_000L
             );
 
     public static final ConfigOption<Long> TRANSPORT_FINISH_SESSION_TIMEOUT =
@@ -797,7 +807,7 @@
                     "it will trigger the sending unavailable if the number " +
                     "of unreceived ack >= max_pending_requests.",
                     positiveInt(),
-                    8000
+                    8
             );
 
     public static final ConfigOption<Integer> TRANSPORT_MIN_PENDING_REQUESTS =
@@ -807,7 +817,7 @@
                     "it will trigger the sending available if the number of " +
                     "unreceived ack < min_pending_requests.",
                     positiveInt(),
-                    6000
+                    6
             );
 
     public static final ConfigOption<Long> TRANSPORT_MIN_ACK_INTERVAL =
@@ -823,7 +833,7 @@
                     "transport.server_idle_timeout",
                     "The max timeout(in ms) of server idle.",
                     positiveInt(),
-                    120_000L
+                    360_000L
             );
 
     public static final ConfigOption<Long> TRANSPORT_HEARTBEAT_INTERVAL =
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/IOFactory.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/IOFactory.java
index 9b626c8..07d87c7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/IOFactory.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/IOFactory.java
@@ -26,8 +26,8 @@
 
 import com.baidu.hugegraph.computer.core.common.ComputerContext;
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
 
 public final class IOFactory {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java
index adb5167..9bd152c 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java
@@ -35,8 +35,8 @@
 import com.baidu.hugegraph.computer.core.graph.properties.Properties;
 import com.baidu.hugegraph.computer.core.graph.value.Value;
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntryReader;
+import com.baidu.hugegraph.computer.core.store.entry.EntryInput;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntryReader;
 
 public class StreamGraphInput implements GraphComputeInput {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java
index 08def17..f25c56f 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java
@@ -30,8 +30,8 @@
 import com.baidu.hugegraph.computer.core.graph.properties.Properties;
 import com.baidu.hugegraph.computer.core.graph.value.Value;
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntryWriter;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntryWriter;
 
 public class StreamGraphOutput implements GraphComputeOutput {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/DataClientManager.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/DataClientManager.java
index ae292b9..adf1403 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/DataClientManager.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/DataClientManager.java
@@ -121,6 +121,7 @@
         @Override
         public void exceptionCaught(TransportException cause,
                                     ConnectionId connectionId) {
+            // TODO: implement failover
             LOG.error("Channel for connectionId {} occurred exception",
                       connectionId, cause);
             DataClientManager.this.connManager.closeClient(connectionId);
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/MessageHandler.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/MessageHandler.java
index 008bcc7..4136b70 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/MessageHandler.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/MessageHandler.java
@@ -19,7 +19,7 @@
 
 package com.baidu.hugegraph.computer.core.network;
 
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 
 public interface MessageHandler extends TransportHandler {
@@ -30,7 +30,12 @@
      * caller if the receiving list reached threshold and the sorting list is
      * sorting in process.
      */
-    void handle(MessageType messageType, int partition, ManagedBuffer buffer);
+    void handle(MessageType messageType, int partition, NetworkBuffer buffer);
+
+    /**
+     * Build a output path.
+     */
+    String genOutputPath(MessageType messageType, int partition);
 
     /**
      * Notify start-session completed on server-side.
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportConf.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportConf.java
index e18d10a..0744373 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportConf.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportConf.java
@@ -80,6 +80,10 @@
                    .createObject(ComputerOptions.TRANSPORT_PROVIDER_CLASS);
     }
 
+    public boolean recvBufferFileMode() {
+        return this.config.get(ComputerOptions.TRANSPORT_RECV_FILE_MODE);
+    }
+
     /**
      * IO mode: nio or epoll
      */
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java
index 0de20fe..2667f5f 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java
@@ -27,16 +27,14 @@
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
-
 import org.slf4j.Logger;
-
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.util.StringEncoding;
 import com.baidu.hugegraph.util.E;
 import com.baidu.hugegraph.util.Log;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
+import io.netty.channel.DefaultMaxBytesRecvByteBufAllocator;
 
 public class TransportUtil {
 
@@ -157,4 +155,13 @@
         buf.writeInt(encoded.length);
         buf.writeBytes(encoded);
     }
+
+    public static void setMaxBytesPerRead(Channel channel, int length) {
+        DefaultMaxBytesRecvByteBufAllocator recvByteBufAllocator =
+                                            channel.config()
+                                                   .getRecvByteBufAllocator();
+        if (recvByteBufAllocator.maxBytesPerIndividualRead() != length) {
+            recvByteBufAllocator.maxBytesPerReadPair(length, length);
+        }
+    }
 }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/FileRegionBuffer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/FileRegionBuffer.java
new file mode 100644
index 0000000..1c0a73c
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/FileRegionBuffer.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.network.buffer;
+
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+
+import com.baidu.hugegraph.computer.core.common.Constants;
+import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
+import com.baidu.hugegraph.computer.core.network.TransportUtil;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.channel.unix.FileDescriptor;
+
+public class FileRegionBuffer implements NetworkBuffer {
+
+    private final int length;
+    private String path;
+
+    public FileRegionBuffer(int length) {
+        this.length = length;
+    }
+
+    /**
+     * Use zero-copy transform from socket channel to file
+     * @param channel
+     * @param targetPath
+     * @return channelFuture
+     */
+    public ChannelFuture transformFromChannel(SocketChannel channel,
+                                              String targetPath) {
+        assert channel.eventLoop().inEventLoop();
+        ChannelPromise channelPromise = channel.newPromise();
+        try {
+            if (channel instanceof EpollSocketChannel) {
+                // Use splice zero-copy if io mode is epoll
+                FileDescriptor fd = FileDescriptor.from(targetPath);
+                try {
+                    ((EpollSocketChannel) channel).spliceTo(fd, 0,
+                                                            this.length,
+                                                            channelPromise);
+                    channelPromise.addListener(future -> fd.close());
+                } catch (Throwable throwable) {
+                    fd.close();
+                    throw throwable;
+                }
+            } else {
+                // Use memory map zero-copy if io mode is not epoll
+                try (RandomAccessFile file = new RandomAccessFile(targetPath,
+                                             Constants.FILE_MODE_WRITE)) {
+                    FileChannel fileChannel = file.getChannel();
+                    NioSocketChannel nioChannel = (NioSocketChannel) channel;
+                    ReadableByteChannel javaChannel = (ReadableByteChannel)
+                                                      nioChannel.unsafe().ch();
+                    fileChannel.transferFrom(javaChannel, 0, this.length);
+                    channelPromise.setSuccess();
+                    fileChannel.close();
+                }
+            }
+            this.path = targetPath;
+        } catch (Throwable throwable) {
+            channelPromise.setFailure(throwable);
+            throw new ComputerException(
+                  "Failed to transform from socket to file, " +
+                  "targetPath:%s, remoteAddress:%s",
+                  throwable, targetPath, TransportUtil.remoteAddress(channel));
+        }
+        return channelPromise;
+    }
+
+    public String path() {
+        return this.path;
+    }
+
+    @Override
+    public int length() {
+        return this.length;
+    }
+
+    @Override
+    public NetworkBuffer retain() {
+        return this;
+    }
+
+    @Override
+    public NetworkBuffer release() {
+        return this;
+    }
+
+    @Override
+    public int referenceCount() {
+        return -1;
+    }
+
+    @Override
+    public ByteBuffer nioByteBuffer() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ByteBuf nettyByteBuf() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] copyToByteArray() {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyManagedBuffer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyBuffer.java
similarity index 91%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyManagedBuffer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyBuffer.java
index 8c66282..641572b 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyManagedBuffer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NettyBuffer.java
@@ -24,11 +24,11 @@
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufUtil;
 
-public class NettyManagedBuffer implements ManagedBuffer {
+public class NettyBuffer implements NetworkBuffer {
 
     private final ByteBuf buf;
 
-    public NettyManagedBuffer(ByteBuf buf) {
+    public NettyBuffer(ByteBuf buf) {
         this.buf = buf;
     }
 
@@ -38,13 +38,13 @@
     }
 
     @Override
-    public ManagedBuffer retain() {
+    public NetworkBuffer retain() {
         this.buf.retain();
         return this;
     }
 
     @Override
-    public ManagedBuffer release() {
+    public NetworkBuffer release() {
         this.buf.release();
         return this;
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBuffer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBuffer.java
similarity index 87%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBuffer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBuffer.java
index 075cf69..2127d4a 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBuffer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBuffer.java
@@ -29,10 +29,11 @@
  *
  * The implementation should specify how the data is provided:
  *
- * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer
- * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf
+ * - {@link NioBuffer}: data backed by a NIO ByteBuffer
+ * - {@link NettyBuffer}: data backed by a Netty ByteBuf
+ * - {@link FileRegionBuffer}: data backed by a File Region
  */
-public interface ManagedBuffer {
+public interface NetworkBuffer {
 
     /**
      * Number of bytes of the data.
@@ -42,13 +43,13 @@
     /**
      * Increase the reference count by one if applicable.
      */
-    ManagedBuffer retain();
+    NetworkBuffer retain();
 
     /**
      * If applicable, decrease the reference count by one and deallocates
      * the buffer if the reference count reaches zero.
      */
-    ManagedBuffer release();
+    NetworkBuffer release();
 
     /**
      * Returns the reference count.
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioManagedBuffer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioBuffer.java
similarity index 91%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioManagedBuffer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioBuffer.java
index 9a10270..2457c03 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioManagedBuffer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NioBuffer.java
@@ -25,12 +25,12 @@
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 
-public class NioManagedBuffer implements ManagedBuffer {
+public class NioBuffer implements NetworkBuffer {
 
     private final ByteBuffer buffer;
     private final AtomicInteger referenceCount;
 
-    public NioManagedBuffer(ByteBuffer buffer) {
+    public NioBuffer(ByteBuffer buffer) {
         this.buffer = buffer;
         this.referenceCount = new AtomicInteger(1);
     }
@@ -41,13 +41,13 @@
     }
 
     @Override
-    public ManagedBuffer retain() {
+    public NetworkBuffer retain() {
         this.referenceCount.incrementAndGet();
         return this;
     }
 
     @Override
-    public ManagedBuffer release() {
+    public NetworkBuffer release() {
         this.referenceCount.decrementAndGet();
         return this;
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/AbstractMessage.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/AbstractMessage.java
index 3d764e1..c7fe079 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/AbstractMessage.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/AbstractMessage.java
@@ -20,7 +20,7 @@
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 
 import io.netty.buffer.ByteBuf;
 
@@ -65,7 +65,7 @@
     private final int sequenceNumber;
     private final int partition;
     private final int bodyLength;
-    private final ManagedBuffer body;
+    private final NetworkBuffer body;
 
     protected AbstractMessage() {
         this(UNKNOWN_SEQ);
@@ -79,16 +79,16 @@
         this(sequenceNumber, partition, null);
     }
 
-    protected AbstractMessage(int sequenceNumber, ManagedBuffer body) {
+    protected AbstractMessage(int sequenceNumber, NetworkBuffer body) {
         this(sequenceNumber, 0, body);
     }
 
-    protected AbstractMessage(ManagedBuffer body) {
+    protected AbstractMessage(NetworkBuffer body) {
         this(UNKNOWN_SEQ, 0, body);
     }
 
     protected AbstractMessage(int sequenceNumber, int partition,
-                              ManagedBuffer body) {
+                              NetworkBuffer body) {
         this.sequenceNumber = sequenceNumber;
         this.partition = partition;
         if (body != null) {
@@ -101,17 +101,17 @@
     }
 
     @Override
-    public ManagedBuffer encode(ByteBuf buf) {
+    public NetworkBuffer encode(ByteBuf buf) {
         this.encodeHeader(buf);
 
         int bodyStart = buf.writerIndex();
-        ManagedBuffer managedBuffer = this.encodeBody(buf);
+        NetworkBuffer networkBuffer = this.encodeBody(buf);
         int bodyEnd = buf.writerIndex();
 
         int bodyLength;
-        if (managedBuffer != null) {
+        if (networkBuffer != null) {
             assert bodyStart == bodyEnd;
-            bodyLength = managedBuffer.length();
+            bodyLength = networkBuffer.length();
         } else {
             bodyLength = bodyEnd - bodyStart;
         }
@@ -123,7 +123,7 @@
         } finally {
             buf.writerIndex(lastWriteIndex);
         }
-        return managedBuffer;
+        return networkBuffer;
     }
 
     /**
@@ -145,7 +145,7 @@
      * Only serializes the body of this message by writing
      * into the given ByteBuf or return the body buffer.
      */
-    protected ManagedBuffer encodeBody(ByteBuf buf) {
+    protected NetworkBuffer encodeBody(ByteBuf buf) {
         return this.body();
     }
 
@@ -165,7 +165,7 @@
     }
 
     @Override
-    public ManagedBuffer body() {
+    public NetworkBuffer body() {
         return this.hasBody() ? this.body : null;
     }
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/DataMessage.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/DataMessage.java
index 82db656..d5843fb 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/DataMessage.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/DataMessage.java
@@ -19,8 +19,9 @@
 
 package com.baidu.hugegraph.computer.core.network.message;
 
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
-import com.baidu.hugegraph.computer.core.network.buffer.NettyManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.FileRegionBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NettyBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.util.E;
 
 import io.netty.buffer.ByteBuf;
@@ -30,7 +31,7 @@
     private final MessageType type;
 
     public DataMessage(MessageType type, int requestId,
-                       int partition, ManagedBuffer data) {
+                       int partition, NetworkBuffer data) {
         super(requestId, partition, data);
         E.checkArgument(requestId > 0,
                         "The data requestId must be > 0, but got %s",
@@ -44,16 +45,27 @@
     }
 
     /**
-     * Decoding uses the given ByteBuf as our data, and will zero-copy it.
+     * Decoding uses the given ByteBuf as our data.
      */
-    public static DataMessage parseFrom(MessageType type, ByteBuf buf) {
+    public static DataMessage parseWithMemoryBuffer(MessageType type,
+                                                    ByteBuf buf) {
         int requestId = buf.readInt();
         int partition = buf.readInt();
 
         int bodyLength = buf.readInt();
         // Slice body and retain it, the readIndex of buf will auto to body end
         ByteBuf bodySlice = buf.readRetainedSlice(bodyLength);
-        ManagedBuffer managedBuffer = new NettyManagedBuffer(bodySlice);
-        return new DataMessage(type, requestId, partition, managedBuffer);
+        NetworkBuffer networkBuffer = new NettyBuffer(bodySlice);
+        return new DataMessage(type, requestId, partition, networkBuffer);
+    }
+
+    public static DataMessage parseWithFileRegion(MessageType type,
+                                                  ByteBuf buf) {
+        int requestId = buf.readInt();
+        int partition = buf.readInt();
+
+        int bodyLength = buf.readInt();
+        return new DataMessage(type, requestId, partition,
+                               new FileRegionBuffer(bodyLength));
     }
 }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/FailMessage.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/FailMessage.java
index a4ddfb7..770ab52 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/FailMessage.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/FailMessage.java
@@ -20,11 +20,12 @@
 package com.baidu.hugegraph.computer.core.network.message;
 
 import com.baidu.hugegraph.computer.core.network.TransportUtil;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.util.E;
 
 import io.netty.buffer.ByteBuf;
 
+@Deprecated
 public class FailMessage extends AbstractMessage implements ResponseMessage {
 
     private final int errorCode;
@@ -43,7 +44,7 @@
     }
 
     @Override
-    protected ManagedBuffer encodeBody(ByteBuf buf) {
+    protected NetworkBuffer encodeBody(ByteBuf buf) {
         buf.writeInt(this.errorCode);
         TransportUtil.writeString(buf, this.message);
         return null;
@@ -66,6 +67,17 @@
         return new FailMessage(failAckId, failCode, failMsg);
     }
 
+    public static int remainingBytes(ByteBuf buf) {
+        buf.markReaderIndex();
+        // Skip ackId
+        buf.skipBytes(Integer.BYTES);
+        // Skip partition
+        buf.skipBytes(Integer.BYTES);
+        int bodyLength = buf.readInt();
+        buf.resetReaderIndex();
+        return bodyLength - buf.readableBytes();
+    }
+
     public String message() {
         return this.message;
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/Message.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/Message.java
index c3d35cc..27891cb 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/Message.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/message/Message.java
@@ -19,7 +19,7 @@
 
 package com.baidu.hugegraph.computer.core.network.message;
 
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 
 import io.netty.buffer.ByteBuf;
 
@@ -30,9 +30,9 @@
      *
      * @param buf {@link ByteBuf} the header buffer, if use zero-copy.
      * Otherwise it will contain header and body.
-     * @return {@link ManagedBuffer} body buffer, if use zero-copy
+     * @return {@link NetworkBuffer} body buffer, if use zero-copy
      */
-    ManagedBuffer encode(ByteBuf buf);
+    NetworkBuffer encode(ByteBuf buf);
 
     /**
      * Used to identify this message type.
@@ -48,7 +48,7 @@
     /**
      * An optional body for the message.
      */
-    ManagedBuffer body();
+    NetworkBuffer body();
 
     /**
      * The message sequence number
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/AbstractNettyHandler.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/AbstractNettyHandler.java
index 2818406..914987b 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/AbstractNettyHandler.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/AbstractNettyHandler.java
@@ -115,6 +115,7 @@
         this.transportHandler().exceptionCaught(exception, connectionId);
     }
 
+    @Deprecated
     protected void ackFailMessage(ChannelHandlerContext ctx, int failId,
                                   int errorCode, String message) {
         long timeout = this.session().conf().writeSocketTimeout();
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyClientHandler.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyClientHandler.java
index e468cba..0a5f0c6 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyClientHandler.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyClientHandler.java
@@ -25,12 +25,10 @@
 import com.baidu.hugegraph.computer.core.network.message.AbstractMessage;
 import com.baidu.hugegraph.computer.core.network.message.AckMessage;
 import com.baidu.hugegraph.computer.core.network.message.DataMessage;
-import com.baidu.hugegraph.computer.core.network.message.FailMessage;
 import com.baidu.hugegraph.computer.core.network.message.FinishMessage;
 import com.baidu.hugegraph.computer.core.network.message.Message;
 import com.baidu.hugegraph.computer.core.network.message.StartMessage;
 import com.baidu.hugegraph.computer.core.network.session.ClientSession;
-import com.google.common.base.Throwables;
 
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
@@ -86,19 +84,6 @@
     }
 
     @Override
-    protected void processFailMessage(ChannelHandlerContext ctx,
-                                      Channel channel,
-                                      FailMessage failMessage) {
-        int failId = failMessage.ackId();
-        if (failId > AbstractMessage.START_SEQ) {
-            this.session().onRecvAck(failId);
-            this.client.checkAndNotifySendAvailable();
-        }
-
-        super.processFailMessage(ctx, channel, failMessage);
-    }
-
-    @Override
     public void channelInactive(ChannelHandlerContext ctx) throws Exception {
         this.transportHandler().onChannelInactive(this.client.connectionId());
         super.channelInactive(ctx);
@@ -117,11 +102,6 @@
                         TransportUtil.remoteAddress(ctx.channel()));
         }
 
-        // Respond fail message to requester
-        this.ackFailMessage(ctx, AbstractMessage.UNKNOWN_SEQ,
-                            exception.errorCode(),
-                            Throwables.getStackTraceAsString(exception));
-
         this.client.clientHandler().exceptionCaught(exception,
                                                     this.client.connectionId());
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyProtocol.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyProtocol.java
index 64e154e..bd91c12 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyProtocol.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyProtocol.java
@@ -26,6 +26,7 @@
 import com.baidu.hugegraph.computer.core.network.netty.codec.FrameDecoder;
 import com.baidu.hugegraph.computer.core.network.netty.codec.MessageDecoder;
 import com.baidu.hugegraph.computer.core.network.netty.codec.MessageEncoder;
+import com.baidu.hugegraph.computer.core.network.netty.codec.PreciseFrameDecoder;
 import com.baidu.hugegraph.computer.core.network.session.ServerSession;
 
 import io.netty.channel.Channel;
@@ -42,8 +43,6 @@
     private static final ChannelHandler SLOT_HANDLER = new SLOT_HANDLER();
     private static final int DISABLE_IDLE_TIME = 0;
 
-    private static final MessageEncoder ENCODER = MessageEncoder.INSTANCE;
-    private static final MessageDecoder DECODER = MessageDecoder.INSTANCE;
     private static final ServerIdleHandler SERVER_IDLE_HANDLER =
                                            new ServerIdleHandler();
     private static final HeartbeatHandler HEART_BEAT_HANDLER =
@@ -63,14 +62,14 @@
      *
      * <pre>
      *     +----------------------+
-     *     | LocalBuffer          |
+     *     | File / Local Buffer  |
      *     +-----------+----------+
-     *                /|\ (2)handle message
+     *                /|\ (2) zero-copy
      * +---------------+---------------------------------------------------+
      * |               |        SERVER CHANNEL PIPELINE                    |
      * |               |                                                   |
      * |    +----------+----------+ (3)write ack +----------------------+  |
-     * |    | ServerHandler       +------------->+ MessageEncoder       |  |
+     * |    | ServerHandler       |------------->| MessageEncoder       |  |
      * |    +----------+----------+              +-----------+----------+  |
      * |              /|\                                 \|/              |
      * |               |                                   |               |
@@ -80,7 +79,7 @@
      * |              /|\                                  |               |
      * |               |                                   |               |
      * |   +-----------+-----------+                       |               |
-     * |   | FrameDecoder          |                       |               |
+     * |   | PreciseFrameDecoder   |                       |               |
      * |   +-----------+-----------+                       |               |
      * |              /|\                                  |               |
      * +---------------+-----------------------------------+---------------+
@@ -97,11 +96,15 @@
                                             MessageHandler handler) {
         ChannelPipeline pipeline = channel.pipeline();
 
-        pipeline.addLast("encoder", ENCODER);
+        pipeline.addLast("encoder", MessageEncoder.INSTANCE);
 
-        pipeline.addLast("frameDecoder", new FrameDecoder());
-
-        pipeline.addLast("decoder", DECODER);
+        if (this.conf.recvBufferFileMode()) {
+            pipeline.addLast("frameDecoder", new PreciseFrameDecoder());
+            pipeline.addLast("decoder", MessageDecoder.INSTANCE_FILE_REGION);
+        } else {
+            pipeline.addLast("frameDecoder", new FrameDecoder());
+            pipeline.addLast("decoder", MessageDecoder.INSTANCE_MEMORY_BUFFER);
+        }
 
         pipeline.addLast("serverIdleStateHandler",
                          this.newServerIdleStateHandler());
@@ -151,11 +154,11 @@
     protected void initializeClientPipeline(Channel channel) {
         ChannelPipeline pipeline = channel.pipeline();
 
-        pipeline.addLast("encoder", ENCODER);
+        pipeline.addLast("encoder", MessageEncoder.INSTANCE);
 
         pipeline.addLast("frameDecoder", new FrameDecoder());
 
-        pipeline.addLast("decoder", DECODER);
+        pipeline.addLast("decoder", MessageDecoder.INSTANCE_MEMORY_BUFFER);
 
         pipeline.addLast("clientIdleStateHandler",
                          this.newClientIdleStateHandler());
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyServerHandler.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyServerHandler.java
index aa37b59..f98194c 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyServerHandler.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyServerHandler.java
@@ -20,22 +20,24 @@
 package com.baidu.hugegraph.computer.core.network.netty;
 
 import java.util.concurrent.TimeUnit;
-
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
 import com.baidu.hugegraph.computer.core.network.MessageHandler;
 import com.baidu.hugegraph.computer.core.network.TransportUtil;
+import com.baidu.hugegraph.computer.core.network.buffer.FileRegionBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.AbstractMessage;
 import com.baidu.hugegraph.computer.core.network.message.AckMessage;
 import com.baidu.hugegraph.computer.core.network.message.DataMessage;
 import com.baidu.hugegraph.computer.core.network.message.FinishMessage;
 import com.baidu.hugegraph.computer.core.network.message.StartMessage;
 import com.baidu.hugegraph.computer.core.network.session.ServerSession;
-import com.google.common.base.Throwables;
-
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.EventLoop;
+import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.ScheduledFuture;
 
 public class NettyServerHandler extends AbstractNettyHandler {
@@ -77,20 +79,61 @@
     protected void processDataMessage(ChannelHandlerContext ctx,
                                       Channel channel,
                                       DataMessage dataMessage) {
+        NetworkBuffer body = dataMessage.body();
         try {
             int requestId = dataMessage.requestId();
-
             this.serverSession.onRecvData(requestId);
-
-            this.handler.handle(dataMessage.type(), dataMessage.partition(),
-                                dataMessage.body());
-
-            this.serverSession.onHandledData(requestId);
+            if (body instanceof FileRegionBuffer) {
+                this.processFileRegionBuffer(ctx, channel, dataMessage,
+                                             (FileRegionBuffer) body);
+            } else {
+                this.handler.handle(dataMessage.type(), dataMessage.partition(),
+                                    dataMessage.body());
+                this.serverSession.onHandledData(requestId);
+            }
         } finally {
-            dataMessage.release();
+            body.release();
         }
     }
 
+    private void processFileRegionBuffer(ChannelHandlerContext ctx,
+                                         Channel channel,
+                                         DataMessage dataMessage,
+                                         FileRegionBuffer fileRegionBuffer) {
+        // Optimize Value of max bytes of next read
+        TransportUtil.setMaxBytesPerRead(channel, fileRegionBuffer.length());
+        String outputPath = this.handler.genOutputPath(dataMessage.type(),
+                                                       dataMessage.partition());
+        /*
+         * Submit zero-copy task to EventLoop, it will be executed next time
+         * network data is received.
+         */
+        ChannelFuture channelFuture = fileRegionBuffer.transformFromChannel(
+                                      (SocketChannel) channel, outputPath);
+
+        channelFuture.addListener((ChannelFutureListener) future -> {
+            try {
+                if (future.isSuccess()) {
+                    this.handler.handle(dataMessage.type(),
+                                        dataMessage.partition(),
+                                        dataMessage.body());
+                    this.serverSession.onHandledData(
+                            dataMessage.requestId());
+                } else {
+                    this.exceptionCaught(ctx, future.cause());
+                }
+                // Reset max bytes next read to length of frame
+                TransportUtil.setMaxBytesPerRead(future.channel(),
+                                                 AbstractMessage.HEADER_LENGTH);
+                future.channel().unsafe().recvBufAllocHandle().reset(
+                                 future.channel().config());
+                dataMessage.release();
+            } catch (Throwable throwable) {
+                this.exceptionCaught(ctx, throwable);
+            }
+        });
+    }
+
     @Override
     protected void processAckMessage(ChannelHandlerContext ctx, Channel channel,
                                      AckMessage ackMessage) {
@@ -139,17 +182,6 @@
         this.serverSession.onDataAckSent(ackId);
     }
 
-    @Override
-    protected void ackFailMessage(ChannelHandlerContext ctx, int failId,
-                                  int errorCode, String message) {
-        super.ackFailMessage(ctx, failId, errorCode, message);
-
-        if (failId > AbstractMessage.START_SEQ) {
-            this.serverSession.onHandledData(failId);
-            this.serverSession.onDataAckSent(failId);
-        }
-    }
-
     private void checkAndRespondAck(ChannelHandlerContext ctx) {
         if (this.serverSession.needAckFinish()) {
             this.ackFinishMessage(ctx, this.serverSession.finishId());
@@ -187,11 +219,6 @@
                         TransportUtil.remoteAddress(channel));
         }
 
-        // Respond fail message to requester
-        this.ackFailMessage(ctx, AbstractMessage.UNKNOWN_SEQ,
-                            exception.errorCode(),
-                            Throwables.getStackTraceAsString(exception));
-
         ConnectionId connectionId = TransportUtil.remoteConnectionId(channel);
         this.handler.exceptionCaught(exception, connectionId);
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClient.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClient.java
index aa148af..b9f4715 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClient.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClient.java
@@ -50,7 +50,7 @@
     private final ClientSession session;
     private final long timeoutSyncRequest;
     private final long timeoutFinishSession;
-    private volatile boolean preSendAvailable;
+    private boolean preSendAvailable;
 
     protected NettyTransportClient(Channel channel, ConnectionId connectionId,
                                    NettyClientFactory clientFactory,
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java
index 5f036a8..9cff3d4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java
@@ -128,9 +128,13 @@
         this.bootstrap.childOption(ChannelOption.SO_KEEPALIVE,
                                    this.conf.tcpKeepAlive());
 
+        boolean enableLt = this.conf.epollLevelTriggered();
+        // Must be use level trigger mode on zero-copy mode
+        if (this.conf.recvBufferFileMode()) {
+            enableLt = true;
+        }
         // Enable trigger mode for epoll if need
-        NettyEventLoopUtil.enableTriggeredMode(ioMode,
-                                               this.conf.epollLevelTriggered(),
+        NettyEventLoopUtil.enableTriggeredMode(ioMode, enableLt,
                                                this.bootstrap);
 
         if (this.conf.maxSynBacklog() > 0) {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageDecoder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageDecoder.java
index deedd46..d03a47a 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageDecoder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageDecoder.java
@@ -22,7 +22,6 @@
 import com.baidu.hugegraph.computer.core.common.exception.IllegalArgException;
 import com.baidu.hugegraph.computer.core.network.message.AckMessage;
 import com.baidu.hugegraph.computer.core.network.message.DataMessage;
-import com.baidu.hugegraph.computer.core.network.message.FailMessage;
 import com.baidu.hugegraph.computer.core.network.message.FinishMessage;
 import com.baidu.hugegraph.computer.core.network.message.Message;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
@@ -42,38 +41,48 @@
 @ChannelHandler.Sharable
 public class MessageDecoder extends ChannelInboundHandlerAdapter {
 
-    public static final MessageDecoder INSTANCE = new MessageDecoder();
+    public static final MessageDecoder INSTANCE_FILE_REGION =
+                        new MessageDecoder(true);
+    public static final MessageDecoder INSTANCE_MEMORY_BUFFER =
+                        new MessageDecoder(false);
 
-    private MessageDecoder() {
+    private final boolean fileRegionMode;
+
+    private MessageDecoder(boolean fileRegionMode) {
+        this.fileRegionMode = fileRegionMode;
     }
 
     @Override
     public void channelRead(ChannelHandlerContext ctx, Object msg) {
-        if (!(msg instanceof ByteBuf)) {
-            ctx.fireChannelRead(msg);
-            return;
-        }
-
         ByteBuf buf = (ByteBuf) msg;
+        Message message;
         try {
             MessageType msgType = MessageType.decode(buf);
-            Message decoded = this.decode(msgType, buf);
-            ctx.fireChannelRead(decoded);
+            message = this.decode(ctx, msgType, buf);
+            if (message == null) {
+                return;
+            }
         } finally {
             buf.release();
         }
+
+        ctx.fireChannelRead(message);
     }
 
-    private Message decode(MessageType msgType, ByteBuf in) {
+    private Message decode(ChannelHandlerContext ctx,
+                           MessageType msgType,
+                           ByteBuf in) {
         if (msgType.category() == MessageType.Category.DATA) {
             // Decode data message
-            return DataMessage.parseFrom(msgType, in);
+            if (this.fileRegionMode) {
+                return DataMessage.parseWithFileRegion(msgType, in);
+            } else {
+                return DataMessage.parseWithMemoryBuffer(msgType, in);
+            }
         }
         switch (msgType) {
             case START:
                 return StartMessage.parseFrom(in);
-            case FAIL:
-                return FailMessage.parseFrom(in);
             case ACK:
                 return AckMessage.parseFrom(in);
             case FINISH:
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageEncoder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageEncoder.java
index 54f62a0..69a1c66 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageEncoder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/MessageEncoder.java
@@ -20,7 +20,7 @@
 package com.baidu.hugegraph.computer.core.network.netty.codec;
 
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.AbstractMessage;
 import com.baidu.hugegraph.computer.core.network.message.Message;
 
@@ -64,7 +64,7 @@
         try {
             PromiseCombiner combiner = new PromiseCombiner(ctx.executor());
             bufHeader = allocator.directBuffer(AbstractMessage.HEADER_LENGTH);
-            ManagedBuffer bodyBuffer = message.encode(bufHeader);
+            NetworkBuffer bodyBuffer = message.encode(bufHeader);
             ChannelFuture headerWriteFuture = ctx.write(bufHeader);
             /*
              * Released bufHeader after in ctx.write(), set bufHeader = null
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/PreciseFrameDecoder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/PreciseFrameDecoder.java
new file mode 100644
index 0000000..5ea331a
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/codec/PreciseFrameDecoder.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.network.netty.codec;
+
+import java.util.List;
+import org.slf4j.Logger;
+import com.baidu.hugegraph.computer.core.network.TransportUtil;
+import com.baidu.hugegraph.computer.core.network.message.AbstractMessage;
+import com.baidu.hugegraph.util.Log;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.DefaultMaxBytesRecvByteBufAllocator;
+import io.netty.handler.codec.ByteToMessageDecoder;
+
+/**
+ * The {@link PreciseFrameDecoder} is a frame Decoder that precisely controls
+ * the number of the max bytes per read, the decoder should be chosen when
+ * receiving data using zero-copy.
+ */
+public class PreciseFrameDecoder extends ByteToMessageDecoder {
+
+    private static final Logger LOG = Log.logger(PreciseFrameDecoder.class);
+
+    public PreciseFrameDecoder() {
+        super.setSingleDecode(true);
+    }
+
+    @Override
+    public void channelActive(ChannelHandlerContext ctx) throws Exception {
+        super.channelActive(ctx);
+        DefaultMaxBytesRecvByteBufAllocator recvByteBufAllocator =
+        new DefaultMaxBytesRecvByteBufAllocator(AbstractMessage.HEADER_LENGTH,
+                                                AbstractMessage.HEADER_LENGTH);
+        ctx.channel().config().setRecvByteBufAllocator(recvByteBufAllocator);
+    }
+
+    @Override
+    protected void decode(ChannelHandlerContext ctx, ByteBuf in,
+                          List<Object> out) throws Exception {
+        Object decoded = this.decode(ctx, in);
+        if (decoded != null) {
+            out.add(decoded);
+        }
+    }
+
+    protected ByteBuf decode(ChannelHandlerContext ctx, ByteBuf in) {
+        if (in.readableBytes() < AbstractMessage.HEADER_LENGTH) {
+            int nextMaxBytesRead =
+                AbstractMessage.HEADER_LENGTH - in.readableBytes();
+            TransportUtil.setMaxBytesPerRead(ctx.channel(), nextMaxBytesRead);
+            return null;
+        }
+
+        // reset max bytes next read to length of frame
+        TransportUtil.setMaxBytesPerRead(ctx.channel(),
+                                         AbstractMessage.HEADER_LENGTH);
+
+        assert in.readableBytes() <= AbstractMessage.HEADER_LENGTH;
+
+        ByteBuf buf = in.readRetainedSlice(AbstractMessage.HEADER_LENGTH);
+
+        int magicNumber = buf.readShort();
+        if (magicNumber != AbstractMessage.MAGIC_NUMBER) {
+            LOG.warn("Network stream corrupted: received incorrect " +
+                     "magic number: {}, remote address: {}",
+                     magicNumber, TransportUtil.remoteAddress(ctx.channel()));
+            buf.release();
+            return null;
+        }
+        int version = buf.readByte();
+        if (version != AbstractMessage.PROTOCOL_VERSION) {
+            LOG.warn("Network stream corrupted: received incorrect " +
+                     "protocol version: {}, remote address: {}",
+                     version, TransportUtil.remoteAddress(ctx.channel()));
+            buf.release();
+            return null;
+        }
+
+        return buf;
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java
index c68d8af..23997fc 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java
@@ -35,8 +35,8 @@
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
 import com.baidu.hugegraph.computer.core.network.TransportConf;
 import com.baidu.hugegraph.computer.core.network.TransportState;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
-import com.baidu.hugegraph.computer.core.network.buffer.NioManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NioBuffer;
 import com.baidu.hugegraph.computer.core.network.message.AbstractMessage;
 import com.baidu.hugegraph.computer.core.network.message.DataMessage;
 import com.baidu.hugegraph.computer.core.network.message.FinishMessage;
@@ -181,9 +181,9 @@
                         "at sendAsync()", this.state);
         int requestId = this.nextRequestId();
 
-        ManagedBuffer managedBuffer = new NioManagedBuffer(buffer);
+        NetworkBuffer networkBuffer = new NioBuffer(buffer);
         DataMessage dataMessage = new DataMessage(messageType, requestId,
-                                                  partition, managedBuffer);
+                                                  partition, networkBuffer);
 
         this.sendFunction.apply(dataMessage);
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffers.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffers.java
index 6daca5b..504479c 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffers.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffers.java
@@ -25,7 +25,7 @@
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.concurrent.BarrierEvent;
 
 public class MessageRecvBuffers {
@@ -33,7 +33,7 @@
     /*
      * The bytesLimit is the limit of total bytes in buffers. The bytesLimit
      * is not hard limit. For performance, it checks whether totalBytes >=
-     * bytesLimit after {@link #addBuffer(ManagedBuffer)}.
+     * bytesLimit after {@link #addBuffer(NetworkBuffer)}.
      * If totalBytes >= bytesLimit, the buffers will be sorted into a file.
      */
     private final long bytesLimit;
@@ -51,10 +51,10 @@
         this.buffers = new ArrayList<>();
     }
 
-    public void addBuffer(ManagedBuffer data) {
+    public void addBuffer(NetworkBuffer data) {
         /*
          * TODO: don't not use copy, add a new class
-         *       RandomAccessInput(ManagedBuffer)
+         *       RandomAccessInput(NetworkBuffer)
          */
         byte[] bytes = data.copyToByteArray();
         this.buffers.add(bytes);
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManager.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManager.java
index 70eecb5..c943866 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManager.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManager.java
@@ -34,7 +34,7 @@
 import com.baidu.hugegraph.computer.core.manager.Manager;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
 import com.baidu.hugegraph.computer.core.network.MessageHandler;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.receiver.edge.EdgeMessageRecvPartitions;
 import com.baidu.hugegraph.computer.core.receiver.message.ComputeMessageRecvPartitions;
@@ -43,7 +43,7 @@
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.util.E;
 import com.baidu.hugegraph.util.Log;
 
@@ -142,7 +142,7 @@
     public void exceptionCaught(TransportException cause,
                                 ConnectionId connectionId) {
         // TODO: implement failover
-        LOG.warn("Exception caught for connection:{}, root cause:{}",
+        LOG.warn("Exception caught for connection:{}, root cause:",
                  connectionId, cause);
     }
 
@@ -175,7 +175,7 @@
 
     @Override
     public void handle(MessageType messageType, int partition,
-                       ManagedBuffer buffer) {
+                       NetworkBuffer buffer) {
         switch (messageType) {
             case VERTEX:
                 this.vertexPartitions.addBuffer(partition, buffer);
@@ -188,12 +188,28 @@
                 break;
             default:
                 throw new ComputerException(
-                          "Unable handle ManagedBuffer with type '%s'",
+                          "Unable handle NetworkBuffer with type '%s'",
                           messageType.name());
         }
     }
 
     @Override
+    public String genOutputPath(MessageType messageType, int partition) {
+        switch (messageType) {
+            case VERTEX:
+                return this.vertexPartitions.genOutputPath(partition);
+            case EDGE:
+                return this.edgePartitions.genOutputPath(partition);
+            case MSG:
+                return this.messagePartitions.genOutputPath(partition);
+            default:
+                throw new ComputerException(
+                      "Unable generator output path with type '%s'",
+                      messageType.name());
+        }
+    }
+
+    @Override
     public void onStarted(ConnectionId connectionId) {
         LOG.debug("ConnectionId {} started", connectionId);
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartition.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartition.java
index 70e91c2..63fd665 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartition.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartition.java
@@ -22,18 +22,17 @@
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.slf4j.Logger;
-
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.FileRegionBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.computer.core.util.FileUtil;
 import com.baidu.hugegraph.util.Log;
 
@@ -59,6 +58,7 @@
     private final boolean withSubKv;
     private final int mergeFileNum;
     private long totalBytes;
+    private final boolean useFileRegion;
 
     private final AtomicReference<Throwable> exception;
 
@@ -75,10 +75,15 @@
         long waitSortTimeout = config.get(
                                ComputerOptions.WORKER_WAIT_SORT_TIMEOUT);
         this.mergeFileNum = config.get(ComputerOptions.HGKV_MERGE_FILES_NUM);
-        this.recvBuffers = new MessageRecvBuffers(buffersLimit,
-                                                  waitSortTimeout);
-        this.sortBuffers = new MessageRecvBuffers(buffersLimit,
-                                                  waitSortTimeout);
+        this.useFileRegion = config.get(
+                             ComputerOptions.TRANSPORT_RECV_FILE_MODE);
+        if (!this.useFileRegion) {
+            this.recvBuffers = new MessageRecvBuffers(buffersLimit,
+                                                      waitSortTimeout);
+            this.sortBuffers = new MessageRecvBuffers(buffersLimit,
+                                                      waitSortTimeout);
+        }
+
         this.outputFiles = new ArrayList<>();
         this.totalBytes = 0L;
         this.exception = new AtomicReference<>();
@@ -87,8 +92,13 @@
     /**
      * Only one thread can call this method.
      */
-    public synchronized void addBuffer(ManagedBuffer buffer) {
+    public synchronized void addBuffer(NetworkBuffer buffer) {
         this.totalBytes += buffer.length();
+        if (buffer instanceof FileRegionBuffer) {
+            String path = ((FileRegionBuffer) buffer).path();
+            this.outputFiles.add(path);
+            return;
+        }
         this.recvBuffers.addBuffer(buffer);
         if (this.recvBuffers.full()) {
             // Wait for the previous sorting
@@ -104,7 +114,9 @@
          * TODO: create iterator directly from buffers if there is no
          *       outputFiles.
          */
-        this.flushAllBuffersAndWaitSorted();
+        if (!this.useFileRegion) {
+            this.flushAllBuffersAndWaitSorted();
+        }
         this.mergeOutputFilesIfNeeded();
         if (this.outputFiles.size() == 0) {
             return PeekableIterator.emptyIterator();
@@ -129,7 +141,7 @@
      * Flush the receive buffers to file, and wait both recvBuffers and
      * sortBuffers to finish sorting.
      * After this method be called, can not call
-     * {@link #addBuffer(ManagedBuffer)} any more.
+     * {@link #addBuffer(NetworkBuffer)} any more.
      */
     private void flushAllBuffersAndWaitSorted() {
         this.sortBuffers.waitSorted();
@@ -143,7 +155,7 @@
     }
 
     private void flushSortBuffersAsync() {
-        String path = this.fileGenerator.nextPath(this.type());
+        String path = this.genOutputPath();
         this.mergeBuffersAsync(this.sortBuffers, path);
         this.outputFiles.add(path);
     }
@@ -194,10 +206,14 @@
         this.outputFiles = newOutputs;
     }
 
+    public String genOutputPath() {
+        return this.fileGenerator.nextPath(this.type());
+    }
+
     private List<String> genOutputFileNames(int targetSize) {
         List<String> files = new ArrayList<>(targetSize);
         for (int i = 0; i < targetSize; i++) {
-            files.add(this.fileGenerator.nextPath(this.type()));
+            files.add(this.genOutputPath());
         }
         return files;
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartitions.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartitions.java
index 1296b3e..8fda027 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartitions.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvPartitions.java
@@ -28,11 +28,11 @@
 
 import com.baidu.hugegraph.computer.core.common.ComputerContext;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public abstract class MessageRecvPartitions<P extends MessageRecvPartition> {
 
@@ -56,11 +56,18 @@
 
     protected abstract P createPartition();
 
-    public void addBuffer(int partitionId, ManagedBuffer buffer) {
+    public void addBuffer(int partitionId, NetworkBuffer buffer) {
         P partition = this.partition(partitionId);
         partition.addBuffer(buffer);
     }
 
+    public String genOutputPath(int partitionId) {
+        P partition = this.partition(partitionId);
+        String path = partition.genOutputPath();
+        new File(path).getParentFile().mkdirs();
+        return path;
+    }
+
     private P partition(int partitionId) {
         P partition = this.partitions.get(partitionId);
         if (partition == null) {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffer.java
index 4c37feb..1214ac7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffer.java
@@ -29,8 +29,8 @@
 import com.baidu.hugegraph.computer.core.io.GraphComputeOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.StreamGraphOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
 
 /**
  * It's not a public class, need package access
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/BufferFileSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/BufferFileSorter.java
new file mode 100644
index 0000000..c670aaf
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/BufferFileSorter.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.sort;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.baidu.hugegraph.computer.core.config.Config;
+import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
+import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesWithFirstSubKvInput;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.bufferfile.BufferFileEntryBuilder;
+import com.baidu.hugegraph.computer.core.store.file.bufferfile.BufferFileEntryReader;
+import com.baidu.hugegraph.computer.core.store.file.bufferfile.BufferFileSubEntryReader;
+import com.baidu.hugegraph.computer.core.store.file.select.DisperseEvenlySelector;
+import com.baidu.hugegraph.computer.core.store.file.select.InputFilesSelector;
+import com.baidu.hugegraph.computer.core.store.file.select.SelectedFiles;
+
+public class BufferFileSorter implements Sorter {
+
+    private final DefaultSorter sorter;
+
+    public BufferFileSorter(Config config) {
+        this.sorter = new DefaultSorter(config);
+    }
+
+    @Override
+    public void sortBuffer(RandomAccessInput input, InnerSortFlusher flusher,
+                           boolean withSubKv) throws Exception {
+        try (EntryIterator entries = new KvEntriesInput(input, withSubKv)) {
+            this.sorter.sortBuffer(entries, flusher);
+        }
+    }
+
+    @Override
+    public void mergeBuffers(List<RandomAccessInput> inputs,
+                             OuterSortFlusher flusher, String output,
+                             boolean withSubKv) throws Exception {
+        List<EntryIterator> entries;
+        if (withSubKv) {
+            entries = inputs.stream()
+                            .map(KvEntriesWithFirstSubKvInput::new)
+                            .collect(Collectors.toList());
+        } else {
+            entries = inputs.stream()
+                            .map(KvEntriesInput::new)
+                            .collect(Collectors.toList());
+        }
+        try (KvEntryFileWriter writer = new BufferFileEntryBuilder(output)) {
+            this.sorter.mergeBuffers(entries, writer, flusher);
+        }
+    }
+
+    @Override
+    public void mergeInputs(List<String> inputs, OuterSortFlusher flusher,
+                            List<String> outputs, boolean withSubKv)
+                            throws Exception {
+        Function<String, EntryIterator> fileToInput;
+        Function<String, KvEntryFileWriter> fileToWriter;
+        if (withSubKv) {
+            fileToInput = o -> new BufferFileSubEntryReader(o).iterator();
+        } else {
+            fileToInput = o -> new BufferFileEntryReader(o).iterator();
+        }
+        fileToWriter = BufferFileEntryBuilder::new;
+
+        InputFilesSelector selector = new DisperseEvenlySelector();
+        List<SelectedFiles> selectResult = selector.selectedByBufferFile(
+                                                    inputs, outputs);
+        this.sorter.mergeFile(selectResult, fileToInput, fileToWriter, flusher);
+    }
+
+    @Override
+    public PeekableIterator<KvEntry> iterator(List<String> inputs,
+                                              boolean withSubKv)
+                                              throws IOException {
+        Function<String, EntryIterator> fileToEntries = input -> {
+            return new BufferFileEntryReader(input, withSubKv).iterator();
+        };
+        return this.sorter.iterator(inputs, fileToEntries);
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/DefaultSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/DefaultSorter.java
new file mode 100644
index 0000000..594b07f
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/DefaultSorter.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.sort;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.baidu.hugegraph.computer.core.config.Config;
+import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
+import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIteratorAdaptor;
+import com.baidu.hugegraph.computer.core.sort.merge.FileMerger;
+import com.baidu.hugegraph.computer.core.sort.merge.FileMergerImpl;
+import com.baidu.hugegraph.computer.core.sort.sorter.InputSorter;
+import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorter;
+import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorterImpl;
+import com.baidu.hugegraph.computer.core.sort.sorter.JavaInputSorter;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.select.SelectedFiles;
+
+public class DefaultSorter {
+
+    private final Config config;
+
+    public DefaultSorter(Config config) {
+        this.config = config;
+    }
+
+    public void sortBuffer(EntryIterator entries, InnerSortFlusher flusher)
+                           throws Exception {
+        InputSorter sorter = new JavaInputSorter();
+        flusher.flush(sorter.sort(entries));
+    }
+
+    public void mergeBuffers(List<EntryIterator> entries,
+                             KvEntryFileWriter writer, OuterSortFlusher flusher)
+                             throws IOException {
+        InputsSorter sorter = new InputsSorterImpl();
+        EntryIterator result = sorter.sort(entries);
+        flusher.flush(result, writer);
+    }
+
+    public void mergeFile(List<SelectedFiles> selectedFiles,
+                          Function<String, EntryIterator> fileToEntries,
+                          Function<String, KvEntryFileWriter> fileToWriter,
+                          OuterSortFlusher flusher) throws Exception {
+        FileMerger merger = new FileMergerImpl(this.config);
+        for (SelectedFiles select : selectedFiles) {
+            merger.merge(select.inputs(), fileToEntries,
+                         select.output(), fileToWriter, flusher);
+        }
+    }
+
+    public PeekableIterator<KvEntry> iterator(
+           List<String> inputs, Function<String, EntryIterator> fileToEntries)
+           throws IOException {
+        List<EntryIterator> entries = inputs.stream()
+                                            .map(fileToEntries)
+                                            .collect(Collectors.toList());
+        InputsSorterImpl sorter = new InputsSorterImpl();
+        return PeekableIteratorAdaptor.of(sorter.sort(entries));
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/HgkvFileSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/HgkvFileSorter.java
new file mode 100644
index 0000000..15cbf09
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/HgkvFileSorter.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.sort;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.baidu.hugegraph.computer.core.config.Config;
+import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
+import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
+import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesWithFirstSubKvInput;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvDirBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader.HgkvDir4SubKvReaderImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader.HgkvDirReaderImpl;
+import com.baidu.hugegraph.computer.core.store.file.select.DisperseEvenlySelector;
+import com.baidu.hugegraph.computer.core.store.file.select.InputFilesSelector;
+import com.baidu.hugegraph.computer.core.store.file.select.SelectedFiles;
+
+public class HgkvFileSorter implements Sorter {
+
+    private final Config config;
+    private final DefaultSorter sorter;
+
+    public HgkvFileSorter(Config config) {
+        this.config = config;
+        this.sorter = new DefaultSorter(config);
+    }
+
+    @Override
+    public void sortBuffer(RandomAccessInput input, InnerSortFlusher flusher,
+                           boolean withSubKv) throws Exception {
+        try (EntryIterator entries = new KvEntriesInput(input, withSubKv)) {
+            this.sorter.sortBuffer(entries, flusher);
+        }
+    }
+
+    @Override
+    public void mergeBuffers(List<RandomAccessInput> inputs,
+                             OuterSortFlusher flusher, String output,
+                             boolean withSubKv) throws Exception {
+        List<EntryIterator> entries;
+        if (withSubKv) {
+            entries = inputs.stream()
+                            .map(KvEntriesWithFirstSubKvInput::new)
+                            .collect(Collectors.toList());
+        } else {
+            entries = inputs.stream()
+                            .map(KvEntriesInput::new)
+                            .collect(Collectors.toList());
+        }
+        try (KvEntryFileWriter writer = new HgkvDirBuilderImpl(this.config,
+                                                               output)) {
+            this.sorter.mergeBuffers(entries, writer, flusher);
+        }
+    }
+
+    @Override
+    public void mergeInputs(List<String> inputs, OuterSortFlusher flusher,
+                            List<String> outputs, boolean withSubKv)
+                            throws Exception {
+        Function<String, EntryIterator> fileToInput;
+        Function<String, KvEntryFileWriter> fileToWriter;
+        if (withSubKv) {
+            fileToInput = o -> new HgkvDir4SubKvReaderImpl(o).iterator();
+        } else {
+            fileToInput = o -> new HgkvDirReaderImpl(o).iterator();
+        }
+        fileToWriter = path -> new HgkvDirBuilderImpl(this.config, path);
+
+        InputFilesSelector selector = new DisperseEvenlySelector();
+        List<SelectedFiles> selectResult = selector.selectedByHgkvFile(
+                                                    inputs, outputs);
+        this.sorter.mergeFile(selectResult, fileToInput, fileToWriter, flusher);
+    }
+
+    @Override
+    public PeekableIterator<KvEntry> iterator(List<String> inputs,
+                                              boolean withSubKv)
+                                              throws IOException {
+        Function<String, EntryIterator> fileToEntries = input -> {
+            return new HgkvDirReaderImpl(input, false, withSubKv).iterator();
+        };
+        return this.sorter.iterator(inputs, fileToEntries);
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/Sorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/Sorter.java
index faea965..980b97e 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/Sorter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/Sorter.java
@@ -26,7 +26,7 @@
 import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public interface Sorter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterImpl.java
deleted file mode 100644
index 1891cda..0000000
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterImpl.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Copyright 2017 HugeGraph Authors
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-
-package com.baidu.hugegraph.computer.core.sort;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
-import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
-import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
-import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIteratorAdaptor;
-import com.baidu.hugegraph.computer.core.sort.merge.HgkvDirMerger;
-import com.baidu.hugegraph.computer.core.sort.merge.HgkvDirMergerImpl;
-import com.baidu.hugegraph.computer.core.sort.sorter.InputSorter;
-import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorter;
-import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorterImpl;
-import com.baidu.hugegraph.computer.core.sort.sorter.JavaInputSorter;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesWithFirstSubKvInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InputToEntries;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDir4SubKvReaderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReader;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReaderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.select.DisperseEvenlySelector;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.select.InputFilesSelector;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.select.SelectedFiles;
-
-public class SorterImpl implements Sorter {
-
-    private final Config config;
-
-    public SorterImpl(Config config) {
-        this.config = config;
-    }
-
-    @Override
-    public void sortBuffer(RandomAccessInput input, InnerSortFlusher flusher,
-                           boolean withSubKv) throws Exception {
-        try (EntryIterator entries = new KvEntriesInput(input, withSubKv)) {
-            InputSorter sorter = new JavaInputSorter();
-            flusher.flush(sorter.sort(entries));
-        }
-    }
-
-    @Override
-    public void mergeBuffers(List<RandomAccessInput> inputs,
-                             OuterSortFlusher flusher, String output,
-                             boolean withSubKv) throws Exception {
-        List<EntryIterator> entries;
-        if (withSubKv) {
-            entries = inputs.stream()
-                            .map(KvEntriesWithFirstSubKvInput::new)
-                            .collect(Collectors.toList());
-        } else {
-            entries = inputs.stream()
-                            .map(KvEntriesInput::new)
-                            .collect(Collectors.toList());
-        }
-
-        this.sortBuffers(entries, flusher, output);
-    }
-
-    @Override
-    public void mergeInputs(List<String> inputs, OuterSortFlusher flusher,
-                            List<String> outputs, boolean withSubKv)
-                            throws Exception {
-        InputToEntries inputToEntries;
-        if (withSubKv) {
-            inputToEntries = o -> new HgkvDir4SubKvReaderImpl(o).iterator();
-        } else {
-            inputToEntries = o -> new HgkvDirReaderImpl(o).iterator();
-        }
-        this.mergeInputs(inputs, inputToEntries, flusher, outputs);
-    }
-
-    @Override
-    public PeekableIterator<KvEntry> iterator(List<String> inputs,
-                                              boolean withSubKv)
-                                              throws IOException {
-        InputsSorterImpl sorter = new InputsSorterImpl();
-        List<EntryIterator> entries = new ArrayList<>();
-        for (String input : inputs) {
-            HgkvDirReader reader = new HgkvDirReaderImpl(input, false,
-                                                         withSubKv);
-            entries.add(reader.iterator());
-        }
-        return PeekableIteratorAdaptor.of(sorter.sort(entries));
-    }
-
-    private void sortBuffers(List<EntryIterator> entries,
-                             OuterSortFlusher flusher, String output)
-                             throws IOException {
-        InputsSorter sorter = new InputsSorterImpl();
-        try (HgkvDirBuilder builder = new HgkvDirBuilderImpl(this.config,
-                                                             output)) {
-            EntryIterator result = sorter.sort(entries);
-            flusher.flush(result, builder);
-        }
-    }
-
-    private void mergeInputs(List<String> inputs, InputToEntries inputToEntries,
-                             OuterSortFlusher flusher, List<String> outputs)
-                             throws Exception {
-        InputFilesSelector selector = new DisperseEvenlySelector();
-        // Each SelectedFiles include some input files per output.
-        List<SelectedFiles> results = selector.selectedOfOutputs(inputs,
-                                                                 outputs);
-
-        HgkvDirMerger merger = new HgkvDirMergerImpl(this.config);
-        for (SelectedFiles result : results) {
-            merger.merge(result.inputs(), inputToEntries,
-                         result.output(), flusher);
-        }
-    }
-}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombinableSorterFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombinableSorterFlusher.java
index a896912..c556c9a 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombinableSorterFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombinableSorterFlusher.java
@@ -23,9 +23,9 @@
 import java.util.Iterator;
 
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.DefaultKvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.DefaultKvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.util.E;
 
 public abstract class CombinableSorterFlusher {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvInnerSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvInnerSortFlusher.java
index e30a168..a49c8a4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvInnerSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvInnerSortFlusher.java
@@ -23,7 +23,7 @@
 
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public class CombineKvInnerSortFlusher extends CombinableSorterFlusher
                                        implements InnerSortFlusher {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvOuterSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvOuterSortFlusher.java
index 8dbfafa..80d8536 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvOuterSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineKvOuterSortFlusher.java
@@ -22,14 +22,14 @@
 import java.io.IOException;
 
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 
 public class CombineKvOuterSortFlusher extends CombinableSorterFlusher
                                        implements OuterSortFlusher {
 
-    private HgkvDirBuilder writer;
+    private KvEntryFileWriter writer;
 
     public CombineKvOuterSortFlusher(PointerCombiner combiner) {
         super(combiner);
@@ -41,7 +41,7 @@
     }
 
     @Override
-    public void flush(EntryIterator entries, HgkvDirBuilder writer)
+    public void flush(EntryIterator entries, KvEntryFileWriter writer)
                       throws IOException {
         this.writer = writer;
         this.flush(entries);
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvInnerSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvInnerSortFlusher.java
index cf82517..12f759f 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvInnerSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvInnerSortFlusher.java
@@ -29,9 +29,9 @@
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortingFactory;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.util.E;
 
 public class CombineSubKvInnerSortFlusher implements InnerSortFlusher {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvOuterSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvOuterSortFlusher.java
index ef055e4..8233d9b 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvOuterSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/CombineSubKvOuterSortFlusher.java
@@ -27,11 +27,11 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.sort.sorter.SubKvSorter;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 import com.baidu.hugegraph.util.E;
 
 public class CombineSubKvOuterSortFlusher implements OuterSortFlusher {
@@ -55,7 +55,7 @@
     }
 
     @Override
-    public void flush(EntryIterator entries, HgkvDirBuilder writer)
+    public void flush(EntryIterator entries, KvEntryFileWriter writer)
                       throws IOException {
         E.checkArgument(entries.hasNext(), "Parameter entries can't be empty");
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/InnerSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/InnerSortFlusher.java
index db14cc2..0b90ade 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/InnerSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/InnerSortFlusher.java
@@ -26,7 +26,7 @@
 
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public interface InnerSortFlusher {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvInnerSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvInnerSortFlusher.java
index 93556ef..f296e38 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvInnerSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvInnerSortFlusher.java
@@ -23,7 +23,7 @@
 import java.util.Iterator;
 
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.util.E;
 
 public class KvInnerSortFlusher implements InnerSortFlusher {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvOuterSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvOuterSortFlusher.java
index 56cf62e..6f9d0a8 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvOuterSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/KvOuterSortFlusher.java
@@ -21,15 +21,15 @@
 
 import java.io.IOException;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 import com.baidu.hugegraph.util.E;
 
 public class KvOuterSortFlusher implements OuterSortFlusher {
 
     @Override
-    public void flush(EntryIterator entries, HgkvDirBuilder writer)
+    public void flush(EntryIterator entries, KvEntryFileWriter writer)
                       throws IOException {
         E.checkArgument(entries.hasNext(),
                         "Parameter entries can't be empty");
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/OuterSortFlusher.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/OuterSortFlusher.java
index 85b91ec..357f80c 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/OuterSortFlusher.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/flusher/OuterSortFlusher.java
@@ -24,9 +24,9 @@
 import org.apache.commons.lang.NotImplementedException;
 
 import com.baidu.hugegraph.computer.core.combiner.Combiner;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 
 public interface OuterSortFlusher {
 
@@ -51,6 +51,6 @@
      * The caller maybe needs to call the sources method before call this
      * method.
      */
-    void flush(EntryIterator entries, HgkvDirBuilder writer)
+    void flush(EntryIterator entries, KvEntryFileWriter writer)
                throws IOException;
 }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMerger.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMerger.java
similarity index 69%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMerger.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMerger.java
index bafa199..851eb6d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMerger.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMerger.java
@@ -20,19 +20,23 @@
 package com.baidu.hugegraph.computer.core.sort.merge;
 
 import java.util.List;
+import java.util.function.Function;
 
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InputToEntries;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 
-public interface HgkvDirMerger {
+public interface FileMerger {
 
     /**
-     * Merge inputs hgkvDirs to output hgkvDir
-     * @param inputs hgkv file that need to be merged
+     * Merge inputs file to output file
+     * @param inputs file that need to be merged
      * @param inputToEntries key value pair read mode
      * @param output write merge result to this file
      * @param flusher combiner entries of same key
      */
-    void merge(List<String> inputs, InputToEntries inputToEntries,
-               String output, OuterSortFlusher flusher) throws Exception;
+    void merge(List<String> inputs,
+               Function<String, EntryIterator> inputToEntries,
+               String output, Function<String, KvEntryFileWriter> fileToWriter,
+               OuterSortFlusher flusher) throws Exception;
 }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMergerImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMergerImpl.java
similarity index 70%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMergerImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMergerImpl.java
index 5449b8e..412b5e4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/HgkvDirMergerImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/merge/FileMergerImpl.java
@@ -23,6 +23,7 @@
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
@@ -30,22 +31,18 @@
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorterImpl;
 import com.baidu.hugegraph.computer.core.sort.sorter.InputsSorter;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InputToEntries;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 import com.baidu.hugegraph.computer.core.util.FileUtil;
 import com.baidu.hugegraph.util.E;
 
-public class HgkvDirMergerImpl implements HgkvDirMerger {
+public class FileMergerImpl implements FileMerger {
 
-    private final Config config;
     private final int mergePathNum;
     private final String tempDir;
 
-    public HgkvDirMergerImpl(Config config) {
-        this.config = config;
+    public FileMergerImpl(Config config) {
         this.mergePathNum = config.get(ComputerOptions.HGKV_MERGE_FILES_NUM);
         this.tempDir = config.get(ComputerOptions.HGKV_TEMP_DIR) +
                        File.separator + UUID.randomUUID();
@@ -55,9 +52,11 @@
     }
 
     @Override
-    public void merge(List<String> inputs, InputToEntries inputToEntries,
-                      String output, OuterSortFlusher flusher)
-                      throws Exception {
+    public void merge(List<String> inputs,
+                      Function<String, EntryIterator> inputToEntries,
+                      String output,
+                      Function<String, KvEntryFileWriter> fileToWriter,
+                      OuterSortFlusher flusher) throws Exception {
         List<String> subInputs = new ArrayList<>(this.mergePathNum);
         int round = 0;
         while (inputs.size() > this.mergePathNum) {
@@ -67,7 +66,8 @@
                 if (subInputs.size() == this.mergePathNum ||
                     i == inputs.size() - 1) {
                     String subOutput = this.mergeInputsToRandomFile(
-                                            subInputs, inputToEntries, flusher);
+                            subInputs, inputToEntries,
+                            fileToWriter, flusher);
                     // Don't remove original file
                     if (round != 0) {
                         FileUtil.deleteFilesQuietly(subInputs);
@@ -81,36 +81,37 @@
             round++;
         }
 
-        this.mergeInputs(inputs, inputToEntries, flusher, output);
+        this.mergeInputs(inputs, inputToEntries, flusher, output, fileToWriter);
     }
 
-    private String mergeInputsToRandomFile(List<String> inputs,
-                                           InputToEntries inputToEntries,
-                                           OuterSortFlusher flusher)
-                                           throws Exception {
+    private String mergeInputsToRandomFile(
+                   List<String> inputs,
+                   Function<String, EntryIterator> inputToIter,
+                   Function<String, KvEntryFileWriter> fileToWriter,
+                   OuterSortFlusher flusher) throws Exception {
         String output = this.randomPath();
-        this.mergeInputs(inputs, inputToEntries, flusher, output);
+        this.mergeInputs(inputs, inputToIter, flusher, output, fileToWriter);
         return output;
     }
 
-    private void mergeInputs(List<String> inputs, InputToEntries inputToEntries,
-                             OuterSortFlusher flusher, String output)
+    private void mergeInputs(List<String> inputs,
+                             Function<String, EntryIterator> inputToIter,
+                             OuterSortFlusher flusher, String output,
+                             Function<String, KvEntryFileWriter> fileToWriter)
                              throws Exception {
         /*
          * File value format is different, upper layer is required to
          * provide the file reading mode
          */
         List<EntryIterator> entries = inputs.stream()
-                                            .map(inputToEntries::inputToEntries)
+                                            .map(inputToIter)
                                             .collect(Collectors.toList());
 
         InputsSorter sorter = new InputsSorterImpl();
-
         // Merge inputs and write to output
         try (EntryIterator sortedKv = sorter.sort(entries);
-             HgkvDirBuilder builder = new HgkvDirBuilderImpl(this.config,
-                                                             output)) {
-            flusher.flush(sortedKv, builder);
+             KvEntryFileWriter builder = fileToWriter.apply(output)) {
+             flusher.flush(sortedKv, builder);
         }
     }
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputSorter.java
index f97c3f3..edfc1a7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputSorter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputSorter.java
@@ -21,7 +21,7 @@
 
 import java.io.IOException;
 import java.util.Iterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public interface InputSorter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorter.java
index 359a66d..eb969c2 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorter.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import java.util.List;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 
 public interface InputsSorter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorterImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorterImpl.java
index 9ea4519..f3f6dc5 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorterImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/InputsSorterImpl.java
@@ -24,8 +24,8 @@
 
 import com.baidu.hugegraph.computer.core.sort.sorting.InputsSorting;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortingFactory;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 
 public class InputsSorterImpl implements InputsSorter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/JavaInputSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/JavaInputSorter.java
index 818d4ec..5cd96e4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/JavaInputSorter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/JavaInputSorter.java
@@ -24,7 +24,7 @@
 import java.util.Iterator;
 import java.util.List;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public class JavaInputSorter implements InputSorter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SubKvSorter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SubKvSorter.java
index 00a0c26..f5f2e93 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SubKvSorter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SubKvSorter.java
@@ -26,8 +26,8 @@
 
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.LoserTreeInputsSorting;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.util.E;
 
 public class SubKvSorter implements Iterator<KvEntry> {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/RecvSortManager.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/RecvSortManager.java
index b1e7a74..c414e8d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/RecvSortManager.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/RecvSortManager.java
@@ -44,6 +44,9 @@
 
     @Override
     protected Integer threadNum(Config config) {
+        if (config.get(ComputerOptions.TRANSPORT_RECV_FILE_MODE)) {
+            return 0;
+        }
         return Math.min(super.threadNum(config),
                         this.maxSendSortThreads(config));
     }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/SortManager.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/SortManager.java
index 8c761bf..600c234 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/SortManager.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/sort/sorting/SortManager.java
@@ -46,15 +46,16 @@
 import com.baidu.hugegraph.computer.core.manager.Manager;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.sender.WriteBuffers;
+import com.baidu.hugegraph.computer.core.sort.BufferFileSorter;
+import com.baidu.hugegraph.computer.core.sort.HgkvFileSorter;
 import com.baidu.hugegraph.computer.core.sort.Sorter;
-import com.baidu.hugegraph.computer.core.sort.SorterImpl;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineSubKvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.KvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.util.ExecutorUtil;
 import com.baidu.hugegraph.util.Log;
 
@@ -71,9 +72,17 @@
     public SortManager(ComputerContext context) {
         this.context = context;
         Config config = context.config();
-        this.sortExecutor = ExecutorUtil.newFixedThreadPool(
-                            this.threadNum(config), this.threadPrefix());
-        this.sorter = new SorterImpl(config);
+        if (this.threadNum(config) != 0) {
+            this.sortExecutor = ExecutorUtil.newFixedThreadPool(
+                                this.threadNum(config), this.threadPrefix());
+        } else {
+            this.sortExecutor = null;
+        }
+        if (config.get(ComputerOptions.TRANSPORT_RECV_FILE_MODE)) {
+            this.sorter = new BufferFileSorter(config);
+        } else {
+            this.sorter = new HgkvFileSorter(config);
+        }
         this.capacity = config.get(
                         ComputerOptions.WORKER_WRITE_BUFFER_INIT_CAPACITY);
         this.flushThreshold = config.get(
@@ -96,6 +105,9 @@
 
     @Override
     public void close(Config config) {
+        if (this.sortExecutor == null) {
+            return;
+        }
         this.sortExecutor.shutdown();
         try {
             this.sortExecutor.awaitTermination(Constants.SHUTDOWN_TIMEOUT,
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/EntryIterator.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/EntryIterator.java
similarity index 89%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/EntryIterator.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/EntryIterator.java
index e70b50e..f942a62 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/EntryIterator.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/EntryIterator.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.buffer;
+package com.baidu.hugegraph.computer.core.store;
 
 import org.apache.commons.lang.NotImplementedException;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.iterator.CIter;
 
 public interface EntryIterator extends CIter<KvEntry> {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileReader.java
new file mode 100644
index 0000000..526073e
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileReader.java
@@ -0,0 +1,6 @@
+package com.baidu.hugegraph.computer.core.store;
+
+public interface KvEntryFileReader {
+
+    EntryIterator iterator();
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileWriter.java
similarity index 80%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilder.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileWriter.java
index 3e74aad..799a3e7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/KvEntryFileWriter.java
@@ -17,22 +17,22 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store;
 
 import java.io.Closeable;
 import java.io.IOException;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
-public interface HgkvDirBuilder extends Closeable {
+public interface KvEntryFileWriter extends Closeable {
 
     /**
-     * Write kvEntry to hgkvDir.
+     * Write kvEntry to file.
      */
     void write(KvEntry entry) throws IOException;
 
     /**
-     * Finish build hgkvDir.
+     * Finish build file.
      * You can release resources or do some aftercare in this method.
      */
     void finish() throws IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesInput.java
similarity index 89%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesInput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesInput.java
index bc64656..7703d58 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesInput.java
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.buffer;
+package com.baidu.hugegraph.computer.core.store.buffer;
 
 import java.io.IOException;
 
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
 
 public class KvEntriesInput implements EntryIterator {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesWithFirstSubKvInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesWithFirstSubKvInput.java
similarity index 86%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesWithFirstSubKvInput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesWithFirstSubKvInput.java
index 77ccfb7..58c25df 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/KvEntriesWithFirstSubKvInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/KvEntriesWithFirstSubKvInput.java
@@ -17,11 +17,12 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.buffer;
+package com.baidu.hugegraph.computer.core.store.buffer;
 
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
 import com.baidu.hugegraph.iterator.CIter;
 import com.baidu.hugegraph.iterator.MapperIterator;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/SubKvEntriesInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/SubKvEntriesInput.java
similarity index 90%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/SubKvEntriesInput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/SubKvEntriesInput.java
index 13aad74..2a05b35 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/buffer/SubKvEntriesInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/buffer/SubKvEntriesInput.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.buffer;
+package com.baidu.hugegraph.computer.core.store.buffer;
 
 import java.io.IOException;
 import java.util.NoSuchElementException;
@@ -25,8 +25,9 @@
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public class SubKvEntriesInput implements EntryIterator {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/AbstractKvEntry.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/AbstractKvEntry.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/AbstractKvEntry.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/AbstractKvEntry.java
index 0020238..5c6d5f7 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/AbstractKvEntry.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/AbstractKvEntry.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 public abstract class AbstractKvEntry implements KvEntry {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/CachedPointer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/CachedPointer.java
similarity index 97%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/CachedPointer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/CachedPointer.java
index 840b6fc..cac6047 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/CachedPointer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/CachedPointer.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/DefaultKvEntry.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/DefaultKvEntry.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/DefaultKvEntry.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/DefaultKvEntry.java
index dc92bc4..8b78330 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/DefaultKvEntry.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/DefaultKvEntry.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 public class DefaultKvEntry extends AbstractKvEntry {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntriesUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntriesUtil.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntriesUtil.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntriesUtil.java
index 7872e64..254e4e0 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntriesUtil.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntriesUtil.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
@@ -26,8 +26,8 @@
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.SubKvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.buffer.SubKvEntriesInput;
 
 public final class EntriesUtil {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInput.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInput.java
index aa19fd1..7cb51e6 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInput.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInputImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInputImpl.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInputImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInputImpl.java
index b1ee6a9..fefe432 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryInputImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryInputImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutput.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutput.java
index 6f2a670..6c7820e 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutput.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutputImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutputImpl.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutputImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutputImpl.java
index 37d4b70..7369d37 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/EntryOutputImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/EntryOutputImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InlinePointer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/InlinePointer.java
similarity index 97%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InlinePointer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/InlinePointer.java
index a37dd03..e95dcfb 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InlinePointer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/InlinePointer.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntry.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntry.java
similarity index 93%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntry.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntry.java
index 05f8d66..862bb9d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntry.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntry.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 public interface KvEntry extends Comparable<KvEntry> {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReader.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReader.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReader.java
index 504e323..73919e1 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReader.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReaderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReaderImpl.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReaderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReaderImpl.java
index 69d733d..57afac0 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryReaderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryReaderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWithFirstSubKv.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWithFirstSubKv.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWithFirstSubKv.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWithFirstSubKv.java
index f72b378..dad2fe4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWithFirstSubKv.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWithFirstSubKv.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 public class KvEntryWithFirstSubKv extends DefaultKvEntry {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriter.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriter.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriter.java
index 214588d..e53a552 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriter.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriterImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriterImpl.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriterImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriterImpl.java
index 32d9c99..52fc7cc 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/KvEntryWriterImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/KvEntryWriterImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -31,7 +31,7 @@
 import com.baidu.hugegraph.computer.core.io.Writable;
 import com.baidu.hugegraph.computer.core.sort.sorter.InputSorter;
 import com.baidu.hugegraph.computer.core.sort.sorter.JavaInputSorter;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
 
 public class KvEntryWriterImpl implements KvEntryWriter {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Pointer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Pointer.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Pointer.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Pointer.java
index 6ed0d21..b90e388 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Pointer.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Pointer.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Range.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Range.java
similarity index 93%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Range.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Range.java
index 1ad146f..0f47bad 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/Range.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/entry/Range.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
+package com.baidu.hugegraph.computer.core.store.entry;
 
 public interface Range {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryBuilder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryBuilder.java
new file mode 100644
index 0000000..799c810
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryBuilder.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.store.file.bufferfile;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
+import com.baidu.hugegraph.computer.core.io.IOFactory;
+import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+
+public class BufferFileEntryBuilder implements KvEntryFileWriter {
+
+    private final RandomAccessOutput output;
+
+    public BufferFileEntryBuilder(String path) {
+        try {
+            this.output = IOFactory.createFileOutput(new File(path));
+        } catch (IOException e) {
+            throw new ComputerException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void write(KvEntry entry) throws IOException {
+        entry.key().write(this.output);
+        entry.value().write(this.output);
+    }
+
+    @Override
+    public void finish() throws IOException {
+        this.close();
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.output.close();
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryReader.java
new file mode 100644
index 0000000..d8bdfc1
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileEntryReader.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.store.file.bufferfile;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
+import com.baidu.hugegraph.computer.core.io.IOFactory;
+import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+
+public class BufferFileEntryReader implements KvEntryFileReader {
+
+    private final File file;
+    private final boolean withSubKv;
+
+    public BufferFileEntryReader(String path, boolean withSubKv) {
+        this.file = new File(path);
+        this.withSubKv = withSubKv;
+    }
+
+    public BufferFileEntryReader(String path) {
+        this(path, false);
+    }
+
+    @Override
+    public EntryIterator iterator() {
+        return new EntryIter();
+    }
+
+    private class EntryIter implements EntryIterator {
+
+        public final RandomAccessInput input;
+        private final RandomAccessInput userAccessInput;
+
+        public EntryIter() {
+            try {
+                this.input = IOFactory.createFileInput(
+                                       BufferFileEntryReader.this.file);
+                this.userAccessInput = this.input.duplicate();
+            } catch (IOException e) {
+                throw new ComputerException(e.getMessage(), e);
+            }
+        }
+
+        @Override
+        public void close() throws Exception {
+            this.input.close();
+            this.userAccessInput.close();
+        }
+
+        @Override
+        public boolean hasNext() {
+            try {
+                return this.input.available() > 0;
+            } catch (IOException e) {
+                throw new ComputerException(e.getMessage(), e);
+            }
+        }
+
+        @Override
+        public KvEntry next() {
+            return EntriesUtil.kvEntryFromInput(
+                               this.input, this.userAccessInput, true,
+                               BufferFileEntryReader.this.withSubKv);
+        }
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileSubEntryReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileSubEntryReader.java
new file mode 100644
index 0000000..5657a7b
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/bufferfile/BufferFileSubEntryReader.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.core.store.file.bufferfile;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
+import com.baidu.hugegraph.computer.core.io.IOFactory;
+import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+
+public class BufferFileSubEntryReader implements KvEntryFileReader {
+
+    private final File file;
+
+    public BufferFileSubEntryReader(String path) {
+        this.file = new File(path);
+    }
+
+    @Override
+    public EntryIterator iterator() {
+        return new BufferFileSubEntryReader.EntryIter();
+    }
+
+    private class EntryIter implements EntryIterator {
+
+        public final RandomAccessInput input;
+        private final RandomAccessInput userAccessInput;
+
+        public EntryIter() {
+            try {
+                this.input = IOFactory.createFileInput(
+                             BufferFileSubEntryReader.this.file);
+                this.userAccessInput = this.input.duplicate();
+            } catch (IOException e) {
+                throw new ComputerException(e.getMessage(), e);
+            }
+        }
+
+        @Override
+        public void close() throws Exception {
+            this.input.close();
+            this.userAccessInput.close();
+        }
+
+        @Override
+        public boolean hasNext() {
+            try {
+                return this.input.available() > 0;
+            } catch (IOException e) {
+                throw new ComputerException(e.getMessage(), e);
+            }
+        }
+
+        @Override
+        public KvEntry next() {
+            return EntriesUtil.subKvEntryFromInput(this.input,
+                                                   this.userAccessInput,
+                                                   true);
+        }
+    }
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/AbstractHgkvFile.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/AbstractHgkvFile.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/AbstractHgkvFile.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/AbstractHgkvFile.java
index 3ffe127..9fd4383 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/AbstractHgkvFile.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/AbstractHgkvFile.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile;
 
 public abstract class AbstractHgkvFile implements HgkvFile {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDir.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDir.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDir.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDir.java
index ba809bd..cb5f2ae 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDir.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDir.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile;
 
 import java.util.List;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDirImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDirImpl.java
similarity index 98%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDirImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDirImpl.java
index 116c6b8..8c3d5ce 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvDirImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvDirImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile;
 
 import java.io.File;
 import java.io.FileNotFoundException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFile.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFile.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFile.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFile.java
index 5f8d6c1..5a0cbbd 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFile.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFile.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile;
 
 import java.io.Closeable;
 import java.io.IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFileImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFileImpl.java
similarity index 97%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFileImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFileImpl.java
index d89f958..5863b00 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/HgkvFileImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/HgkvFileImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile;
 
 import java.io.File;
 import java.io.IOException;
@@ -112,7 +112,7 @@
 
         // Read magic
         String magic = new String(input.readBytes(MAGIC.length()));
-        E.checkArgument(HgkvFileImpl.MAGIC.equals(magic),
+        E.checkArgument(MAGIC.equals(magic),
                         "Failed to read footer, illegal hgvk-file magic in " +
                         "file: '%s'", this.path);
         this.magic = magic;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/BlockBuilder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/BlockBuilder.java
similarity index 89%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/BlockBuilder.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/BlockBuilder.java
index 5a90b66..ccaf56e 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/BlockBuilder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/BlockBuilder.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public interface BlockBuilder {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/DataBlockBuilderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/DataBlockBuilderImpl.java
similarity index 93%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/DataBlockBuilderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/DataBlockBuilderImpl.java
index da735e5..f4179de 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/DataBlockBuilderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/DataBlockBuilderImpl.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public class DataBlockBuilderImpl implements BlockBuilder {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvDirBuilderImpl.java
similarity index 90%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvDirBuilderImpl.java
index 79ac227..cc77720 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvDirBuilderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvDirBuilderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 import java.nio.file.Paths;
@@ -27,12 +27,13 @@
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
 import com.baidu.hugegraph.util.E;
 
-public class HgkvDirBuilderImpl implements HgkvDirBuilder {
+public class HgkvDirBuilderImpl implements KvEntryFileWriter {
 
     private final Config config;
     // The max byte size of hgkv-file data
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilder.java
similarity index 91%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilder.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilder.java
index 8ddf7db..45c2a87 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilder.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.Closeable;
 import java.io.IOException;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 
 public interface HgkvFileBuilder extends Closeable {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilderImpl.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilderImpl.java
index 38f2dc9..c06cd6e 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/HgkvFileBuilderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/HgkvFileBuilderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -26,9 +26,9 @@
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFile;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFileImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFile;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFileImpl;
 import com.baidu.hugegraph.util.E;
 
 public class HgkvFileBuilderImpl implements HgkvFileBuilder {
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilder.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilder.java
similarity index 93%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilder.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilder.java
index 13f3560..e2b2f82 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilder.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilder.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilderImpl.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilderImpl.java
index 67fe96e..c911acf 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/builder/IndexBlockBuilderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/builder/IndexBlockBuilderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder;
 
 import java.io.IOException;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDir4SubKvReaderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDir4SubKvReaderImpl.java
similarity index 79%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDir4SubKvReaderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDir4SubKvReaderImpl.java
index 2c20933..bf101b3 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDir4SubKvReaderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDir4SubKvReaderImpl.java
@@ -17,17 +17,18 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
 import com.baidu.hugegraph.iterator.CIter;
 import com.baidu.hugegraph.iterator.MapperIterator;
 
-public class HgkvDir4SubKvReaderImpl implements HgkvDirReader {
+public class HgkvDir4SubKvReaderImpl implements KvEntryFileReader {
 
-    private final HgkvDirReader reader;
+    private final KvEntryFileReader reader;
 
     public HgkvDir4SubKvReaderImpl(String path, boolean useInlinePointer) {
         this.reader = new HgkvDirReaderImpl(path, useInlinePointer, true);
@@ -46,7 +47,7 @@
 
         private final CIter<KvEntry> entries;
 
-        public KvEntryWithFirstSubKvIter(HgkvDirReader reader) {
+        public KvEntryWithFirstSubKvIter(KvEntryFileReader reader) {
             this.entries = new MapperIterator<>(reader.iterator(), entry -> {
                 return EntriesUtil.kvEntryWithFirstSubKv(entry);
             });
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReaderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDirReaderImpl.java
similarity index 85%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReaderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDirReaderImpl.java
index 481af36..99a1416 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReaderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvDirReaderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +25,14 @@
 import java.util.NoSuchElementException;
 
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFile;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFile;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 
-public class HgkvDirReaderImpl implements HgkvDirReader {
+public class HgkvDirReaderImpl implements KvEntryFileReader {
 
     private final HgkvDir hgkvDir;
     private final boolean useInlinePointer;
@@ -122,9 +123,9 @@
 
         private EntryIterator nextKeyIter() throws Exception {
             HgkvFile segment = this.segments.get(this.segmentIndex++);
-            HgkvFileReader reader = new HgkvFileReaderImpl(
-                                    segment.path(), this.useInlinePointer,
-                                    this.withSubKv);
+            KvEntryFileReader reader = new HgkvFileReaderImpl(
+                                       segment.path(), this.useInlinePointer,
+                                       this.withSubKv);
             EntryIterator iterator = reader.iterator();
             this.segmentsIters.add(iterator);
             return iterator;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReaderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvFileReaderImpl.java
similarity index 77%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReaderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvFileReaderImpl.java
index 4bc5ba1..badfdfc 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReaderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/hgkvfile/reader/HgkvFileReaderImpl.java
@@ -17,21 +17,23 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader;
+package com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.NoSuchElementException;
 
+import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFile;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFileImpl;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFile;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFileImpl;
 
-public class HgkvFileReaderImpl implements HgkvFileReader {
+public class HgkvFileReaderImpl implements KvEntryFileReader {
 
     private final HgkvFile hgkvFile;
     private final boolean useInlinePointer;
@@ -51,7 +53,7 @@
     }
 
     @Override
-    public EntryIterator iterator() throws IOException {
+    public EntryIterator iterator() {
         return new EntryIter(this.hgkvFile, this.useInlinePointer,
                              this.withSubKv);
     }
@@ -66,13 +68,16 @@
         private final boolean withSubKv;
 
         public EntryIter(HgkvFile hgkvFile, boolean useInlinePointer,
-                         boolean withSubKv)
-                         throws IOException {
+                         boolean withSubKv) {
             this.file = hgkvFile;
             this.numEntries = this.file.numEntries();
             File file = new File(this.file.path());
-            this.input = IOFactory.createFileInput(file);
-            this.userAccessInput = this.input.duplicate();
+            try {
+                this.input = IOFactory.createFileInput(file);
+                this.userAccessInput = this.input.duplicate();
+            } catch (IOException e) {
+                throw new ComputerException(e.getMessage(), e);
+            }
             this.useInlinePointer = useInlinePointer;
             this.withSubKv = withSubKv;
         }
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DefaultSelectedFiles.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DefaultSelectedFiles.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DefaultSelectedFiles.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DefaultSelectedFiles.java
index 675166a..159edfa 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DefaultSelectedFiles.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DefaultSelectedFiles.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.select;
+package com.baidu.hugegraph.computer.core.store.file.select;
 
 import java.util.List;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DisperseEvenlySelector.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DisperseEvenlySelector.java
similarity index 78%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DisperseEvenlySelector.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DisperseEvenlySelector.java
index 2520078..4bf7af4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/DisperseEvenlySelector.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/DisperseEvenlySelector.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.select;
+package com.baidu.hugegraph.computer.core.store.file.select;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,18 +25,18 @@
 import java.util.List;
 import java.util.stream.Collectors;
 
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFile;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFile;
 import com.baidu.hugegraph.util.E;
 import com.google.common.collect.Lists;
 
 public class DisperseEvenlySelector implements InputFilesSelector {
 
     @Override
-    public List<SelectedFiles> selectedOfOutputs(List<String> inputs,
-                                                 List<String> outputs)
-                                                 throws IOException {
+    public List<SelectedFiles> selectedByHgkvFile(List<String> inputs,
+                                                  List<String> outputs)
+                                                  throws IOException {
         E.checkArgument(inputs.size() >= outputs.size(),
                         "The inputs size of InputFilesSelector must be >= " +
                         "outputs size, but got %s inputs < %s outputs",
@@ -86,6 +86,32 @@
         return results;
     }
 
+    @Override
+    public List<SelectedFiles> selectedByBufferFile(List<String> inputs,
+                                                    List<String> outputs) {
+        E.checkArgument(inputs.size() >= outputs.size(),
+                        "The inputs size of InputFilesSelector must be >= " +
+                        "outputs size, but got %s inputs < %s outputs",
+                        inputs.size(), outputs.size());
+
+        // TODO: design a better way of distribute
+        int size = outputs.size();
+        List<List<String>> group = new ArrayList<>(size);
+        for (int i = 0; i < size; i++) {
+            group.add(new ArrayList<>());
+        }
+        for (int i = 0; i < inputs.size(); i++) {
+            List<String> item = group.get(i % size);
+            item.add(inputs.get(i));
+        }
+        List<SelectedFiles> result = new ArrayList<>();
+        for (int i = 0; i < size; i++) {
+            result.add(new DefaultSelectedFiles(outputs.get(i), group.get(i)));
+        }
+
+        return result;
+    }
+
     private static class Heap<T> {
 
         private final List<T> data;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/InputFilesSelector.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/InputFilesSelector.java
new file mode 100644
index 0000000..ebff19f
--- /dev/null
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/InputFilesSelector.java
@@ -0,0 +1,23 @@
+package com.baidu.hugegraph.computer.core.store.file.select;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface InputFilesSelector {
+
+    /**
+     * Select the input files to the output files.
+     * @return key is output file. value is input files.
+     */
+    List<SelectedFiles> selectedByHgkvFile(List<String> inputs,
+                                           List<String> outputs)
+                                           throws IOException;
+
+    /**
+     * Select the input files to the output files.
+     * @return key is output file. value is input files.
+     */
+    List<SelectedFiles> selectedByBufferFile(List<String> inputs,
+                                             List<String> outputs)
+                                             throws IOException;
+}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/SelectedFiles.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/SelectedFiles.java
similarity index 92%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/SelectedFiles.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/SelectedFiles.java
index cc1e312..0fdfc05 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/SelectedFiles.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/select/SelectedFiles.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.select;
+package com.baidu.hugegraph.computer.core.store.file.select;
 
 import java.util.List;
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReader.java
similarity index 94%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReader.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReader.java
index 0c1a2ba..6396c17 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReader.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.Closeable;
 import java.io.IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReaderImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReaderImpl.java
similarity index 96%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReaderImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReaderImpl.java
index dc6de4a..61d37b4 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileReaderImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileReaderImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriter.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriter.java
similarity index 95%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriter.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriter.java
index c5d1167..ec7b289 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriter.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriter.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.Closeable;
 import java.io.IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriterImpl.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriterImpl.java
similarity index 97%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriterImpl.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriterImpl.java
index ccd4d0a..30eea3d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/BitsFileWriterImpl.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/BitsFileWriterImpl.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFile.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFile.java
similarity index 97%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFile.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFile.java
index 7ad2f70..db77c99 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFile.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFile.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.File;
 import java.nio.file.Paths;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileInput.java
similarity index 98%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileInput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileInput.java
index d432b9a..41e034f 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileInput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileInput.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.EOFException;
 import java.io.File;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileOutput.java
similarity index 98%
rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileOutput.java
rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileOutput.java
index a8c3c13..355ba6d 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/seqfile/ValueFileOutput.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/file/seqfile/ValueFileOutput.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.core.store.seqfile;
+package com.baidu.hugegraph.computer.core.store.file.seqfile;
 
 import java.io.EOFException;
 import java.io.File;
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InputToEntries.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InputToEntries.java
deleted file mode 100644
index 14a4541..0000000
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/entry/InputToEntries.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2017 HugeGraph Authors
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-
-package com.baidu.hugegraph.computer.core.store.hgkvfile.entry;
-
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-
-public interface InputToEntries {
-
-    EntryIterator inputToEntries(String path);
-}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReader.java
deleted file mode 100644
index e859d35..0000000
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvDirReader.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2017 HugeGraph Authors
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader;
-
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-
-public interface HgkvDirReader {
-
-    EntryIterator iterator();
-}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReader.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReader.java
deleted file mode 100644
index 878ea33..0000000
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/reader/HgkvFileReader.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2017 HugeGraph Authors
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader;
-
-import java.io.IOException;
-
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-
-public interface HgkvFileReader {
-
-    EntryIterator iterator() throws IOException;
-}
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/InputFilesSelector.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/InputFilesSelector.java
deleted file mode 100644
index b2d9b53..0000000
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/store/hgkvfile/file/select/InputFilesSelector.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package com.baidu.hugegraph.computer.core.store.hgkvfile.file.select;
-
-import java.io.IOException;
-import java.util.List;
-
-public interface InputFilesSelector {
-
-    /**
-     * Select the input files to the output files.
-     * @return key is output file. value is input files.
-     */
-    List<SelectedFiles> selectedOfOutputs(List<String> inputs,
-                                          List<String> outputs)
-                                          throws IOException;
-}
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombinerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombinerTest.java
index a85d9f2..8cb827b 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombinerTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/combiner/PointerCombinerTest.java
@@ -37,8 +37,8 @@
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InlinePointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.InlinePointer;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManagerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManagerTest.java
index 88fbd80..8ff418f 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManagerTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/ComputeManagerTest.java
@@ -48,7 +48,7 @@
 import com.baidu.hugegraph.computer.core.io.StreamGraphOutput;
 import com.baidu.hugegraph.computer.core.manager.Managers;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.receiver.MessageRecvManager;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
@@ -56,8 +56,8 @@
 import com.baidu.hugegraph.computer.core.sort.sorting.SendSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Whitebox;
 
@@ -73,23 +73,24 @@
     @Before
     public void setup() {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "2",
-            ComputerOptions.BSP_MAX_SUPER_STEP, "2",
-            ComputerOptions.WORKER_COMBINER_CLASS,
-            Null.class.getName(), // Can't combine
-            ComputerOptions.ALGORITHM_RESULT_CLASS,
-            IdListList.class.getName(),
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS,
-            IdList.class.getName(),
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
-            ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
-            ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
-            ComputerOptions.WORKER_COMPUTATION_CLASS,
-            MockComputation.class.getName(),
-            ComputerOptions.INPUT_EDGE_FREQ, "SINGLE"
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "2",
+                ComputerOptions.BSP_MAX_SUPER_STEP, "2",
+                ComputerOptions.WORKER_COMBINER_CLASS,
+                Null.class.getName(), // Can't combine
+                ComputerOptions.ALGORITHM_RESULT_CLASS,
+                IdListList.class.getName(),
+                ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                IdList.class.getName(),
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
+                ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
+                ComputerOptions.WORKER_COMPUTATION_CLASS,
+                MockComputation.class.getName(),
+                ComputerOptions.INPUT_EDGE_FREQ, "SINGLE",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
 
         this.managers = new Managers();
@@ -123,16 +124,16 @@
         MessageRecvManager receiveManager = this.managers.get(
                                             MessageRecvManager.NAME);
         receiveManager.onStarted(this.connectionId);
-        add200VertexBuffer((ManagedBuffer buffer) -> {
+        add200VertexBuffer((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.VERTEX, 0, buffer);
         });
         // Partition 1 only has vertex.
-        add200VertexBuffer((ManagedBuffer buffer) -> {
+        add200VertexBuffer((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.VERTEX, 1, buffer);
         });
         receiveManager.onFinished(this.connectionId);
         receiveManager.onStarted(this.connectionId);
-        addSingleFreqEdgeBuffer((ManagedBuffer buffer) -> {
+        addSingleFreqEdgeBuffer((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.EDGE, 0, buffer);
         });
         receiveManager.onFinished(this.connectionId);
@@ -141,7 +142,7 @@
         // Superstep 0
         receiveManager.beforeSuperstep(this.config, 0);
         receiveManager.onStarted(this.connectionId);
-        addMessages((ManagedBuffer buffer) -> {
+        addMessages((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.MSG, 0, buffer);
         });
         receiveManager.onFinished(this.connectionId);
@@ -160,7 +161,7 @@
         this.computeManager.output();
     }
 
-    private static void add200VertexBuffer(Consumer<ManagedBuffer> consumer)
+    private static void add200VertexBuffer(Consumer<NetworkBuffer> consumer)
                                            throws IOException {
         for (long i = 0L; i < 200L; i += 2) {
             Vertex vertex = graphFactory().createVertex();
@@ -181,7 +182,7 @@
     }
 
     private static void addSingleFreqEdgeBuffer(
-                        Consumer<ManagedBuffer> consumer) throws IOException {
+                        Consumer<NetworkBuffer> consumer) throws IOException {
         for (long i = 0L; i < 200L; i++) {
             Vertex vertex = graphFactory().createVertex();
             vertex.id(BytesId.of(i));
@@ -217,7 +218,7 @@
         return bytesOutput.toByteArray();
     }
 
-    private static void addMessages(Consumer<ManagedBuffer> consumer)
+    private static void addMessages(Consumer<NetworkBuffer> consumer)
                                     throws IOException {
         for (long i = 0L; i < 200L; i++) {
             int count = RANDOM.nextInt(5);
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/EdgesInputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/EdgesInputTest.java
index 11625b4..9b58285 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/EdgesInputTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/EdgesInputTest.java
@@ -51,7 +51,7 @@
 import com.baidu.hugegraph.computer.core.io.StreamGraphOutput;
 import com.baidu.hugegraph.computer.core.manager.Managers;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.receiver.MessageRecvManager;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
@@ -60,8 +60,8 @@
 import com.baidu.hugegraph.computer.core.sort.sorting.SendSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.testutil.Whitebox;
@@ -110,20 +110,21 @@
     private void testEdgeFreq(EdgeFrequency freq)
                               throws IOException {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "1",
-            ComputerOptions.WORKER_COMBINER_CLASS,
-            Null.class.getName(), // Can't combine
-            ComputerOptions.ALGORITHM_RESULT_CLASS,
-            IdListList.class.getName(),
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS,
-            IdList.class.getName(),
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
-            ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
-            ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
-            ComputerOptions.INPUT_EDGE_FREQ, freq.name()
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "1",
+                ComputerOptions.WORKER_COMBINER_CLASS,
+                Null.class.getName(), // Can't combine
+                ComputerOptions.ALGORITHM_RESULT_CLASS,
+                IdListList.class.getName(),
+                ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                IdList.class.getName(),
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
+                ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
+                ComputerOptions.INPUT_EDGE_FREQ, freq.name(),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         this.managers = new Managers();
         FileManager fileManager = new FileManager();
@@ -146,12 +147,12 @@
         FileGraphPartition partition = new FileGraphPartition(
                                            context(), this.managers, 0);
         receiveManager.onStarted(connectionId);
-        add200VertexBuffer((ManagedBuffer buffer) -> {
+        add200VertexBuffer((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.VERTEX, 0, buffer);
         });
         receiveManager.onFinished(connectionId);
         receiveManager.onStarted(connectionId);
-        addEdgeBuffer((ManagedBuffer buffer) -> {
+        addEdgeBuffer((NetworkBuffer buffer) -> {
             receiveManager.handle(MessageType.EDGE, 0, buffer);
         }, freq);
 
@@ -168,7 +169,7 @@
         edgesInput.close();
     }
 
-    private static void add200VertexBuffer(Consumer<ManagedBuffer> consumer)
+    private static void add200VertexBuffer(Consumer<NetworkBuffer> consumer)
                                            throws IOException {
         for (long i = 0L; i < 200L; i += 2) {
             Vertex vertex = graphFactory().createVertex();
@@ -188,7 +189,7 @@
         return bytesOutput.toByteArray();
     }
 
-    private static void addEdgeBuffer(Consumer<ManagedBuffer> consumer,
+    private static void addEdgeBuffer(Consumer<NetworkBuffer> consumer,
                                       EdgeFrequency freq) throws IOException {
         for (long i = 0L; i < 200L; i++) {
             Vertex vertex = graphFactory().createVertex();
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInputTest.java
index 053c8fe..69ffd1e 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInputTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/compute/input/MessageInputTest.java
@@ -43,7 +43,7 @@
 import com.baidu.hugegraph.computer.core.graph.value.IdListList;
 import com.baidu.hugegraph.computer.core.manager.Managers;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.receiver.MessageRecvManager;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
@@ -51,7 +51,7 @@
 import com.baidu.hugegraph.computer.core.sort.sorting.RecvSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
@@ -64,22 +64,23 @@
     @Before
     public void setup() {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "2",
-            ComputerOptions.BSP_MAX_SUPER_STEP, "2",
-            ComputerOptions.WORKER_COMBINER_CLASS,
-            Null.class.getName(), // Can't combine
-            ComputerOptions.ALGORITHM_RESULT_CLASS,
-            IdListList.class.getName(),
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS,
-            IdList.class.getName(),
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
-            ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
-            ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
-            ComputerOptions.WORKER_COMPUTATION_CLASS,
-            MockComputation.class.getName()
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "2",
+                ComputerOptions.BSP_MAX_SUPER_STEP, "2",
+                ComputerOptions.WORKER_COMBINER_CLASS,
+                Null.class.getName(), // Can't combine
+                ComputerOptions.ALGORITHM_RESULT_CLASS,
+                IdListList.class.getName(),
+                ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                IdList.class.getName(),
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
+                ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "1000",
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "10",
+                ComputerOptions.WORKER_COMPUTATION_CLASS,
+                MockComputation.class.getName(),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
 
         this.managers = new Managers();
@@ -112,7 +113,7 @@
         // Superstep 0
         receiveManager.beforeSuperstep(this.config, 0);
         receiveManager.onStarted(this.connectionId);
-        addMessages((ManagedBuffer buffer) -> {
+        addMessages((NetworkBuffer buffer) -> {
                     receiveManager.handle(MessageType.MSG, 0, buffer);
         });
         receiveManager.onFinished(this.connectionId);
@@ -141,7 +142,7 @@
         }
     }
 
-    private static void addMessages(Consumer<ManagedBuffer> consumer)
+    private static void addMessages(Consumer<NetworkBuffer> consumer)
                                     throws IOException {
         Random random = new Random(1);
         for (long i = 0L; i < 200L; i++) {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java
index 30520ff..0f22781 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java
@@ -38,7 +38,7 @@
 import org.slf4j.Logger;
 
 import com.baidu.hugegraph.computer.core.common.Constants;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.util.Log;
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockClientHandler.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockClientHandler.java
index a59af02..59b710b 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockClientHandler.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockClientHandler.java
@@ -42,7 +42,7 @@
     public void exceptionCaught(TransportException cause,
                                 ConnectionId connectionId) {
         // Close the client associated with the given connectionId
-        LOG.info("Client connection exception, connectionId: {}, cause: {}",
+        LOG.info("Client connection exception, connectionId: {}, cause:",
                  connectionId, cause);
     }
 
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockMessageHandler.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockMessageHandler.java
index 422ebb8..da73a30 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockMessageHandler.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/MockMessageHandler.java
@@ -19,10 +19,15 @@
 
 package com.baidu.hugegraph.computer.core.network;
 
+import java.io.File;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.FileRegionBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.util.Log;
 
@@ -32,17 +37,28 @@
 
     @Override
     public void handle(MessageType messageType, int partition,
-                       ManagedBuffer buffer) {
+                       NetworkBuffer buffer) {
         LOG.info("Receive data from remote, messageType: {}, partition: {}, " +
                  "buffer readable length: {}", messageType.name(), partition,
                  buffer != null ? buffer.length() : null);
 
         if (buffer != null) {
-            buffer.copyToByteArray();
+            if (buffer instanceof FileRegionBuffer) {
+                String path = ((FileRegionBuffer) buffer).path();
+                LOG.info("path: {}", path);
+                FileUtils.deleteQuietly(new File(path));
+            } else {
+                buffer.copyToByteArray();
+            }
         }
     }
 
     @Override
+    public String genOutputPath(MessageType messageType, int partition) {
+        return "./" + UUID.randomUUID().toString();
+    }
+
+    @Override
     public void onStarted(ConnectionId connectionId) {
         LOG.info("Start session completed, connectionId: {}",
                  connectionId);
@@ -67,7 +83,7 @@
     @Override
     public void exceptionCaught(TransportException cause,
                                 ConnectionId connectionId) {
-        LOG.info("Server channel exception, connectionId: {}, cause: {}",
+        LOG.info("Server channel exception, connectionId: {}, cause:",
                  connectionId, cause);
     }
 }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/NetworkTestSuite.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/NetworkTestSuite.java
index efaaca5..9ed71d0 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/NetworkTestSuite.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/NetworkTestSuite.java
@@ -22,7 +22,7 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBufferTest;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBufferTest;
 import com.baidu.hugegraph.computer.core.network.connection.ConnectionManagerTest;
 import com.baidu.hugegraph.computer.core.network.netty.HeartbeatHandlerTest;
 import com.baidu.hugegraph.computer.core.network.netty.NettyClientFactoryTest;
@@ -42,7 +42,7 @@
     NettyTransportClientTest.class,
     NettyEncodeDecodeHandlerTest.class,
     HeartbeatHandlerTest.class,
-    ManagedBufferTest.class,
+    NetworkBufferTest.class,
     DataServerManagerTest.class
 })
 public class NetworkTestSuite {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBufferTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBufferTest.java
similarity index 66%
rename from computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBufferTest.java
rename to computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBufferTest.java
index 0ba0aa0..ad2b441 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/ManagedBufferTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/buffer/NetworkBufferTest.java
@@ -29,74 +29,74 @@
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 
-public class ManagedBufferTest {
+public class NetworkBufferTest {
 
     @Test
     public void testRetain() {
         ByteBuffer byteBuffer = ByteBuffer.allocate(10);
-        ManagedBuffer nioManagedBuffer = new NioManagedBuffer(byteBuffer);
-        nioManagedBuffer.retain();
-        nioManagedBuffer.release();
-        Assert.assertSame(nioManagedBuffer.nioByteBuffer().array(),
+        NetworkBuffer nioNetworkBuffer = new NioBuffer(byteBuffer);
+        nioNetworkBuffer.retain();
+        nioNetworkBuffer.release();
+        Assert.assertSame(nioNetworkBuffer.nioByteBuffer().array(),
                           byteBuffer.array());
-        nioManagedBuffer.release();
+        nioNetworkBuffer.release();
 
         ByteBuf byteBuf = Unpooled.buffer(10);
         int cnt = byteBuf.refCnt();
-        ManagedBuffer nettyManagedBuffer = new NettyManagedBuffer(byteBuf);
-        nettyManagedBuffer.retain();
+        NetworkBuffer nettyNetworkBuffer = new NettyBuffer(byteBuf);
+        nettyNetworkBuffer.retain();
         Assert.assertSame(cnt + 1, byteBuf.refCnt());
-        Assert.assertSame(cnt + 1, nettyManagedBuffer.referenceCount());
-        ByteBuf buf = nettyManagedBuffer.nettyByteBuf();
-        nettyManagedBuffer.retain();
+        Assert.assertSame(cnt + 1, nettyNetworkBuffer.referenceCount());
+        ByteBuf buf = nettyNetworkBuffer.nettyByteBuf();
+        nettyNetworkBuffer.retain();
         Assert.assertSame(cnt + 2, buf.refCnt());
-        Assert.assertSame(cnt + 2, nettyManagedBuffer.referenceCount());
-        nettyManagedBuffer.release();
-        nettyManagedBuffer.release();
-        nettyManagedBuffer.release();
+        Assert.assertSame(cnt + 2, nettyNetworkBuffer.referenceCount());
+        nettyNetworkBuffer.release();
+        nettyNetworkBuffer.release();
+        nettyNetworkBuffer.release();
     }
 
     @Test
     public void testRelease() {
         ByteBuffer byteBuffer = ByteBuffer.allocate(10);
-        ManagedBuffer nioManagedBuffer = new NioManagedBuffer(byteBuffer);
-        Assert.assertSame(nioManagedBuffer.nioByteBuffer().array(),
+        NetworkBuffer nioNetworkBuffer = new NioBuffer(byteBuffer);
+        Assert.assertSame(nioNetworkBuffer.nioByteBuffer().array(),
                           byteBuffer.array());
-        nioManagedBuffer.release();
+        nioNetworkBuffer.release();
 
         ByteBuf byteBuf = Unpooled.buffer(10);
         int cnt = byteBuf.refCnt();
-        ManagedBuffer nettyManagedBuffer = new NettyManagedBuffer(byteBuf);
-        nettyManagedBuffer.release();
-        Assert.assertSame(cnt - 1, nettyManagedBuffer.referenceCount());
+        NetworkBuffer nettyNetworkBuffer = new NettyBuffer(byteBuf);
+        nettyNetworkBuffer.release();
+        Assert.assertSame(cnt - 1, nettyNetworkBuffer.referenceCount());
     }
 
     @Test
     public void testNioByteBuffer() {
         ByteBuffer byteBuffer = ByteBuffer.allocate(10);
-        ManagedBuffer nioManagedBuffer = new NioManagedBuffer(byteBuffer);
-        Assert.assertSame(nioManagedBuffer.nioByteBuffer().array(),
+        NetworkBuffer nioNetworkBuffer = new NioBuffer(byteBuffer);
+        Assert.assertSame(nioNetworkBuffer.nioByteBuffer().array(),
                           byteBuffer.array());
-        nioManagedBuffer.release();
+        nioNetworkBuffer.release();
 
         ByteBuf byteBuf = Unpooled.buffer(10);
-        ManagedBuffer nettyManagedBuffer = new NettyManagedBuffer(byteBuf);
-        ByteBuffer buffer = nettyManagedBuffer.nioByteBuffer();
+        NetworkBuffer nettyNetworkBuffer = new NettyBuffer(byteBuf);
+        ByteBuffer buffer = nettyNetworkBuffer.nioByteBuffer();
         Assert.assertSame(buffer.array(), byteBuf.array());
     }
 
     @Test
     public void testNettyByteBuffer() {
         ByteBuffer byteBuffer = ByteBuffer.allocate(10);
-        ManagedBuffer nioManagedBuffer = new NioManagedBuffer(byteBuffer);
-        Assert.assertSame(nioManagedBuffer.nettyByteBuf().array(),
+        NetworkBuffer nioNetworkBuffer = new NioBuffer(byteBuffer);
+        Assert.assertSame(nioNetworkBuffer.nettyByteBuf().array(),
                           byteBuffer.array());
-        nioManagedBuffer.release();
-        Assert.assertEquals(0, nioManagedBuffer.referenceCount());
+        nioNetworkBuffer.release();
+        Assert.assertEquals(0, nioNetworkBuffer.referenceCount());
 
         ByteBuf byteBuf = Unpooled.buffer(10);
-        ManagedBuffer nettyManagedBuffer = new NettyManagedBuffer(byteBuf);
-        ByteBuf buf = nettyManagedBuffer.nettyByteBuf();
+        NetworkBuffer nettyNetworkBuffer = new NettyBuffer(byteBuf);
+        ByteBuf buf = nettyNetworkBuffer.nettyByteBuf();
         Assert.assertSame(buf.array(), byteBuf.array());
     }
 
@@ -108,7 +108,7 @@
         ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bytesSource.length);
         byteBuffer = byteBuffer.put(bytesSource);
         byteBuffer.flip();
-        NioManagedBuffer nioManagedBuffer = new NioManagedBuffer(byteBuffer);
+        NioBuffer nioManagedBuffer = new NioBuffer(byteBuffer);
         byte[] bytes = nioManagedBuffer.copyToByteArray();
         Assert.assertArrayEquals(bytesSource, bytes);
         Assert.assertNotSame(bytesSource, bytes);
@@ -119,7 +119,7 @@
 
         int position = byteBuffer2.position();
         int remaining = byteBuffer2.remaining();
-        NioManagedBuffer nioManagedBuffer2 = new NioManagedBuffer(byteBuffer2);
+        NioBuffer nioManagedBuffer2 = new NioBuffer(byteBuffer2);
         byte[] bytes2 = nioManagedBuffer2.copyToByteArray();
         Assert.assertArrayEquals(bytesSource, bytes2);
         Assert.assertNotSame(bytesSource, bytes2);
@@ -131,8 +131,8 @@
             buf3 = buf3.writeBytes(bytesSource);
             int readerIndex = buf3.readerIndex();
             int readableBytes = buf3.readableBytes();
-            NettyManagedBuffer nettyManagedBuffer3 =
-                               new NettyManagedBuffer(buf3);
+            NettyBuffer nettyManagedBuffer3 =
+                               new NettyBuffer(buf3);
             byte[] bytes3 = nettyManagedBuffer3.copyToByteArray();
             Assert.assertArrayEquals(bytesSource, bytes3);
             Assert.assertNotSame(bytesSource, bytes3);
@@ -145,8 +145,8 @@
         ByteBuf buf4 = Unpooled.buffer(bytesSource.length);
         try {
             buf4 = buf4.writeBytes(bytesSource);
-            NettyManagedBuffer nettyManagedBuffer4 =
-                               new NettyManagedBuffer(buf4);
+            NettyBuffer nettyManagedBuffer4 =
+                               new NettyBuffer(buf4);
             byte[] bytes4 = nettyManagedBuffer4.copyToByteArray();
             Assert.assertArrayEquals(bytesSource, bytes4);
             Assert.assertNotSame(bytesSource, bytes4);
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyEncodeDecodeHandlerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyEncodeDecodeHandlerTest.java
index 142357d..3e6de20 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyEncodeDecodeHandlerTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyEncodeDecodeHandlerTest.java
@@ -25,16 +25,16 @@
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.computer.core.network.MockUnDecodeMessage;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
-import com.baidu.hugegraph.computer.core.network.buffer.NettyManagedBuffer;
-import com.baidu.hugegraph.computer.core.network.buffer.NioManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NettyBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NioBuffer;
 import com.baidu.hugegraph.computer.core.network.message.DataMessage;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.network.message.StartMessage;
 import com.baidu.hugegraph.computer.core.network.netty.codec.FrameDecoder;
 import com.baidu.hugegraph.computer.core.util.StringEncoding;
+import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
 import io.netty.buffer.ByteBuf;
@@ -57,7 +57,7 @@
         int partition = 1;
         byte[] bytes = StringEncoding.encode("mock msg");
         ByteBuffer buffer = ByteBuffer.wrap(bytes);
-        ManagedBuffer body = new NioManagedBuffer(buffer);
+        NetworkBuffer body = new NioBuffer(buffer);
         DataMessage dataMessage = new DataMessage(null, requestId,
                                                   partition, body);
         ChannelFutureListenerOnWrite listener =
@@ -89,7 +89,7 @@
     public void testSendMsgWithFrameDecode() {
         FrameDecoder frameDecoder = new FrameDecoder();
         EmbeddedChannel embeddedChannel = new EmbeddedChannel(frameDecoder);
-        ManagedBuffer buffer = new NettyManagedBuffer(Unpooled.buffer());
+        NetworkBuffer buffer = new NettyBuffer(Unpooled.buffer());
         ByteBuf buf = buffer.nettyByteBuf();
         StartMessage.INSTANCE.encode(buf);
         boolean writeInbound = embeddedChannel.writeInbound(buf);
@@ -102,7 +102,7 @@
     public void testSendMsgWithFrameDecodeMagicError() {
         FrameDecoder frameDecoder = new FrameDecoder();
         EmbeddedChannel embeddedChannel = new EmbeddedChannel(frameDecoder);
-        ManagedBuffer buffer = new NettyManagedBuffer(Unpooled.buffer());
+        NetworkBuffer buffer = new NettyBuffer(Unpooled.buffer());
         short magicError = 10;
         ByteBuf buf = buffer.nettyByteBuf();
         StartMessage.INSTANCE.encode(buf);
@@ -117,7 +117,7 @@
     public void testSendMsgWithFrameDecodeVersionError() {
         FrameDecoder frameDecoder = new FrameDecoder();
         EmbeddedChannel embeddedChannel = new EmbeddedChannel(frameDecoder);
-        ManagedBuffer buffer = new NettyManagedBuffer(Unpooled.buffer());
+        NetworkBuffer buffer = new NettyBuffer(Unpooled.buffer());
         byte versionError = 10;
         ByteBuf buf = buffer.nettyByteBuf();
         StartMessage.INSTANCE.encode(buf);
@@ -152,7 +152,7 @@
         byte[] bytes = StringEncoding.encode("mock msg");
         ByteBuf buf = Unpooled.directBuffer().writeBytes(bytes);
         try {
-            NettyManagedBuffer managedBuffer = new NettyManagedBuffer(buf);
+            NettyBuffer managedBuffer = new NettyBuffer(buf);
             DataMessage dataMessage = new DataMessage(MessageType.MSG,
                                                       requestId, partition,
                                                       managedBuffer);
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java
index 1d285dc..3975fd6 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java
@@ -19,6 +19,7 @@
 
 package com.baidu.hugegraph.computer.core.network.netty;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -27,48 +28,38 @@
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
 
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.config.Configurator;
+import org.apache.commons.io.FileUtils;
 import org.junit.Test;
 import org.mockito.Mockito;
-import org.slf4j.Logger;
 
 import com.baidu.hugegraph.computer.core.common.ComputerContext;
-import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
 import com.baidu.hugegraph.computer.core.network.TransportConf;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.FileRegionBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.Message;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.util.StringEncoding;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
-import com.baidu.hugegraph.concurrent.BarrierEvent;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.testutil.Whitebox;
 import com.baidu.hugegraph.util.Bytes;
-import com.baidu.hugegraph.util.Log;
 
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 
 public class NettyTransportClientTest extends AbstractNetworkTest {
 
-    private static final Logger LOG =
-            Log.logger(NettyTransportClientTest.class);
-
-    public static final BarrierEvent BARRIER_EVENT = new BarrierEvent();
-
     @Override
     protected void initOption() {
         super.updateOption(ComputerOptions.TRANSPORT_MAX_PENDING_REQUESTS,
-                           8000);
+                           8);
         super.updateOption(ComputerOptions.TRANSPORT_MIN_PENDING_REQUESTS,
-                           6000);
+                           6);
         super.updateOption(ComputerOptions.TRANSPORT_WRITE_BUFFER_HIGH_MARK,
                            64 * (int) Bytes.MB);
         super.updateOption(ComputerOptions.TRANSPORT_WRITE_BUFFER_LOW_MARK,
@@ -155,7 +146,7 @@
 
         Mockito.doAnswer(invocationOnMock -> {
             MessageType type = invocationOnMock.getArgument(0);
-            ManagedBuffer buffer = invocationOnMock.getArgument(2);
+            NetworkBuffer buffer = invocationOnMock.getArgument(2);
             byte[] sourceBytes = null;
             switch (type) {
                 case MSG:
@@ -169,13 +160,20 @@
                     break;
                 default:
             }
-            byte[] bytes = buffer.copyToByteArray();
+
+            byte[] bytes;
+            if (buffer instanceof FileRegionBuffer) {
+                String path = ((FileRegionBuffer) buffer).path();
+                File file = new File(path);
+                bytes = FileUtils.readFileToByteArray(file);
+                FileUtils.deleteQuietly(file);
+            } else {
+                bytes = buffer.copyToByteArray();
+            }
+
             Assert.assertArrayEquals(sourceBytes, bytes);
             Assert.assertNotSame(sourceBytes, bytes);
 
-            byte[] bytes2 = buffer.copyToByteArray();
-            Assert.assertArrayEquals(sourceBytes, bytes2);
-            Assert.assertNotSame(sourceBytes, bytes2);
             return null;
         }).when(serverHandler).handle(Mockito.any(), Mockito.eq(1),
                                       Mockito.any());
@@ -270,7 +268,7 @@
     @Test
     public void testFlowControl() throws IOException {
         ByteBuffer buffer = ByteBuffer.wrap(
-                            StringEncoding.encode("test data"));
+                StringEncoding.encode("test data"));
         NettyTransportClient client = (NettyTransportClient) this.oneClient();
 
         client.startSession();
@@ -318,12 +316,13 @@
 
         client.startSession();
 
-        Mockito.doThrow(new RuntimeException("test exception"))
-               .when(serverHandler)
-               .handle(Mockito.any(), Mockito.anyInt(), Mockito.any());
+        Mockito.doAnswer(invocationOnMock -> {
+            invocationOnMock.callRealMethod();
+            throw new RuntimeException("test exception");
+        }).when(serverHandler)
+          .handle(Mockito.any(), Mockito.anyInt(), Mockito.any());
 
-        ByteBuffer buffer = ByteBuffer.wrap(
-                            StringEncoding.encode("test data"));
+        ByteBuffer buffer = ByteBuffer.wrap(StringEncoding.encode("test data"));
         boolean send = client.send(MessageType.MSG, 1, buffer);
         Assert.assertTrue(send);
 
@@ -339,66 +338,6 @@
 
         Mockito.verify(serverHandler, Mockito.timeout(10_000L).times(1))
                .exceptionCaught(Mockito.any(), Mockito.any());
-
-        Mockito.verify(clientHandler, Mockito.timeout(10_000L).times(1))
-               .exceptionCaught(Mockito.any(), Mockito.any());
-    }
-
-    @Test
-    public void testTransportPerformance() throws IOException,
-                                                  InterruptedException {
-        Configurator.setAllLevels("com.baidu.hugegraph", Level.INFO);
-        Configurator.setAllLevels("com.baidu.hugegraph.computer.core.network",
-                                  Level.WARN);
-
-        NettyTransportClient client = (NettyTransportClient) this.oneClient();
-        ByteBuffer buffer = ByteBuffer.allocateDirect(50 * 1024);
-
-        AtomicInteger handledCnt = new AtomicInteger(0);
-
-        Mockito.doAnswer(invocationOnMock -> {
-            invocationOnMock.callRealMethod();
-            BARRIER_EVENT.signalAll();
-            return null;
-        }).when(clientHandler).sendAvailable(Mockito.any());
-
-        Mockito.doAnswer(invocationOnMock -> {
-            invocationOnMock.callRealMethod();
-            handledCnt.getAndIncrement();
-            return null;
-        }).when(serverHandler).handle(Mockito.any(), Mockito.anyInt(),
-                                      Mockito.any());
-
-        long preTransport = System.nanoTime();
-
-        client.startSession();
-
-        int dataNum = 209716;
-        long timout = 10_000L;
-        for (int i = 0; i < dataNum; i++) {
-            boolean send = client.send(MessageType.MSG, 1, buffer);
-            if (!send) {
-                LOG.info("Current send unavailable");
-                i--;
-                if (!BARRIER_EVENT.await(timout)) {
-                    throw new ComputerException("Timeout(%sms) to wait " +
-                                                "sendable", timout);
-                }
-                BARRIER_EVENT.reset();
-            }
-        }
-
-        client.finishSession();
-
-        long postTransport = System.nanoTime();
-
-        LOG.info("Transport {} data packets total 10GB, cost {}ms", dataNum,
-                 (postTransport - preTransport) / 1000_000L);
-
-        Assert.assertEquals(dataNum, handledCnt.get());
-
-        Configurator.setAllLevels("com.baidu.hugegraph.computer.core.network",
-                                  Level.INFO);
     }
 
     @Test
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffersTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffersTest.java
index 0525eee..afc3bc7 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffersTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvBuffersTest.java
@@ -29,7 +29,7 @@
 
 import com.baidu.hugegraph.computer.core.common.exception.ComputerException;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.testutil.Assert;
 
 public class MessageRecvBuffersTest {
@@ -180,7 +180,7 @@
     public static void addMockBufferToBuffers(MessageRecvBuffers buffers,
                                               int mockBufferLength) {
         ReceiverUtil.consumeBuffer(new byte[mockBufferLength],
-                                   (ManagedBuffer buffer) -> {
+                                   (NetworkBuffer buffer) -> {
             buffers.addBuffer(buffer);
         });
     }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManagerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManagerTest.java
index c1cb93a..e09ce56 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManagerTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/MessageRecvManagerTest.java
@@ -33,7 +33,7 @@
 import com.baidu.hugegraph.computer.core.config.Config;
 import com.baidu.hugegraph.computer.core.graph.value.DoubleValue;
 import com.baidu.hugegraph.computer.core.network.ConnectionId;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.network.message.MessageType;
 import com.baidu.hugegraph.computer.core.receiver.edge.EdgeMessageRecvPartitionTest;
 import com.baidu.hugegraph.computer.core.receiver.message.ComputeMessageRecvPartitionTest;
@@ -42,7 +42,7 @@
 import com.baidu.hugegraph.computer.core.sort.sorting.RecvSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
@@ -57,15 +57,17 @@
     @Before
     public void setup() {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.BSP_MAX_SUPER_STEP, "1",
-            ComputerOptions.WORKER_COMBINER_CLASS,
-            DoubleValueSumCombiner.class.getName(),
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "100",
-            ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "100",
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS, DoubleValue.class.getName()
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.BSP_MAX_SUPER_STEP, "1",
+                ComputerOptions.WORKER_COMBINER_CLASS,
+                DoubleValueSumCombiner.class.getName(),
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "100",
+                ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT, "100",
+                ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                DoubleValue.class.getName(),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         this.fileManager = new FileManager();
         this.fileManager.init(this.config);
@@ -93,12 +95,12 @@
         this.receiveManager.onStarted(this.connectionId);
         this.receiveManager.onFinished(this.connectionId);
         VertexMessageRecvPartitionTest.addTenVertexBuffer(
-                                       (ManagedBuffer buffer) -> {
+                                       (NetworkBuffer buffer) -> {
             this.receiveManager.handle(MessageType.VERTEX, 0, buffer);
         });
 
         EdgeMessageRecvPartitionTest.addTenEdgeBuffer(
-                                     (ManagedBuffer buffer) -> {
+                                     (NetworkBuffer buffer) -> {
             this.receiveManager.handle(MessageType.EDGE, 0, buffer);
         });
         // Send edge message
@@ -122,7 +124,7 @@
         // Superstep 0
         this.receiveManager.beforeSuperstep(this.config, 0);
         ComputeMessageRecvPartitionTest.addTwentyCombineMessageBuffer(
-                                        (ManagedBuffer buffer) -> {
+                                        (NetworkBuffer buffer) -> {
              this.receiveManager.handle(MessageType.MSG, 0, buffer);
         });
         this.receiveManager.onFinished(this.connectionId);
@@ -141,11 +143,11 @@
     public void testOtherMessageType() {
         Assert.assertThrows(ComputerException.class, () -> {
             ReceiverUtil.consumeBuffer(new byte[100],
-                                       (ManagedBuffer buffer) -> {
+                                       (NetworkBuffer buffer) -> {
                 this.receiveManager.handle(MessageType.ACK, 0, buffer);
             });
         }, e -> {
-            Assert.assertEquals("Unable handle ManagedBuffer with type 'ACK'",
+            Assert.assertEquals("Unable handle NetworkBuffer with type 'ACK'",
                                 e.getMessage());
         });
     }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/ReceiverUtil.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/ReceiverUtil.java
index 92f28ae..efec7a1 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/ReceiverUtil.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/ReceiverUtil.java
@@ -30,11 +30,11 @@
 import com.baidu.hugegraph.computer.core.io.Readable;
 import com.baidu.hugegraph.computer.core.io.StreamGraphInput;
 import com.baidu.hugegraph.computer.core.io.Writable;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
-import com.baidu.hugegraph.computer.core.network.buffer.NettyManagedBuffer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.network.buffer.NettyBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
@@ -42,11 +42,11 @@
 public class ReceiverUtil {
 
     public static void consumeBuffer(byte[] bytes,
-                                     Consumer<ManagedBuffer> consumer) {
+                                     Consumer<NetworkBuffer> consumer) {
         ByteBuf buf = Unpooled.directBuffer(bytes.length);
         try {
             buf = buf.writeBytes(bytes);
-            NettyManagedBuffer buff = new NettyManagedBuffer(buf);
+            NettyBuffer buff = new NettyBuffer(buf);
             consumer.accept(buff);
         } finally {
             buf.release();
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/edge/EdgeMessageRecvPartitionTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/edge/EdgeMessageRecvPartitionTest.java
index 6d89205..d83241b 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/edge/EdgeMessageRecvPartitionTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/edge/EdgeMessageRecvPartitionTest.java
@@ -42,19 +42,19 @@
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.RecvSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 import com.baidu.hugegraph.computer.core.store.FileManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntryWriter;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntryWriter;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
@@ -68,12 +68,13 @@
     @Before
     public void setup() {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "1",
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "100",
-            ComputerOptions.HGKV_MERGE_FILES_NUM, "2"
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "1",
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "100",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "2",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -98,7 +99,7 @@
     public void testEdgeMessageRecvPartition() throws IOException {
         Assert.assertEquals("edge", this.partition.type());
 
-        addTenEdgeBuffer((ManagedBuffer buffer) -> {
+        addTenEdgeBuffer((NetworkBuffer buffer) -> {
             this.partition.addBuffer(buffer);
         });
 
@@ -129,7 +130,8 @@
             ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
             ComputerOptions.HGKV_MERGE_FILES_NUM, "5",
             ComputerOptions.WORKER_EDGE_PROPERTIES_COMBINER_CLASS,
-            MergeNewPropertiesCombiner.class.getName()
+            MergeNewPropertiesCombiner.class.getName(),
+            ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -147,7 +149,7 @@
         checkTenEdgesWithCombinedProperties(this.partition.iterator());
     }
 
-    public static void addTenEdgeBuffer(Consumer<ManagedBuffer> consumer)
+    public static void addTenEdgeBuffer(Consumer<NetworkBuffer> consumer)
                                         throws IOException {
         for (long i = 0L; i < 10L; i++) {
             Vertex vertex = graphFactory().createVertex();
@@ -167,7 +169,7 @@
     }
 
     private static void addTenDuplicateEdgeBuffer(
-                        Consumer<ManagedBuffer> consumer)
+                        Consumer<NetworkBuffer> consumer)
                         throws IOException {
         for (long i = 0L; i < 10L; i++) {
             Vertex vertex = graphFactory().createVertex();
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/message/ComputeMessageRecvPartitionTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/message/ComputeMessageRecvPartitionTest.java
index 2c6745b..b81eab3 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/message/ComputeMessageRecvPartitionTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/message/ComputeMessageRecvPartitionTest.java
@@ -34,14 +34,14 @@
 import com.baidu.hugegraph.computer.core.graph.id.Id;
 import com.baidu.hugegraph.computer.core.graph.value.DoubleValue;
 import com.baidu.hugegraph.computer.core.graph.value.IdList;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.RecvSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
@@ -50,16 +50,18 @@
     @Test
     public void testCombineMessageRecvPartition() throws IOException {
         Config config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "1",
-            // Make sure all buffers within this limit.
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "1000",
-            ComputerOptions.WORKER_COMBINER_CLASS,
-            DoubleValueSumCombiner.class.getName(),
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10",
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS, DoubleValue.class.getName()
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "1",
+                // Make sure all buffers within this limit.
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "1000",
+                ComputerOptions.WORKER_COMBINER_CLASS,
+                DoubleValueSumCombiner.class.getName(),
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10",
+                ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                DoubleValue.class.getName(),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -92,7 +94,8 @@
             Null.class.getName(),
             ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
             ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10",
-            ComputerOptions.ALGORITHM_MESSAGE_CLASS, IdList.class.getName()
+            ComputerOptions.ALGORITHM_MESSAGE_CLASS, IdList.class.getName(),
+            ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -116,7 +119,7 @@
     }
 
     public static void addTwentyCombineMessageBuffer(
-                       Consumer<ManagedBuffer> consumer)
+                       Consumer<NetworkBuffer> consumer)
                        throws IOException {
         for (long i = 0L; i < 10L; i++) {
             for (int j = 0; j < 2; j++) {
@@ -153,7 +156,7 @@
     }
 
     private static void addTwentyDuplicateIdValueListMessageBuffer
-                        (Consumer<ManagedBuffer> consumer)
+                        (Consumer<NetworkBuffer> consumer)
                         throws IOException {
         for (long i = 0L; i < 10L; i++) {
             for (int j = 0; j < 2; j++) {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/vertex/VertexMessageRecvPartitionTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/vertex/VertexMessageRecvPartitionTest.java
index d054dfe..a10a7f4 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/vertex/VertexMessageRecvPartitionTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/receiver/vertex/VertexMessageRecvPartitionTest.java
@@ -42,17 +42,17 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.StreamGraphInput;
-import com.baidu.hugegraph.computer.core.network.buffer.ManagedBuffer;
+import com.baidu.hugegraph.computer.core.network.buffer.NetworkBuffer;
 import com.baidu.hugegraph.computer.core.receiver.ReceiverUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
 import com.baidu.hugegraph.computer.core.sort.sorting.RecvSortManager;
 import com.baidu.hugegraph.computer.core.sort.sorting.SortManager;
 import com.baidu.hugegraph.computer.core.store.FileManager;
 import com.baidu.hugegraph.computer.core.store.SuperstepFileGenerator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 
@@ -66,12 +66,13 @@
     @Before
     public void setup() {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "1",
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "20",
-            ComputerOptions.HGKV_MERGE_FILES_NUM, "5"
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "1",
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "20",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "5",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -107,12 +108,13 @@
     @Test
     public void testOverwriteCombiner() throws IOException {
         this.config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.JOB_ID, "local_001",
-            ComputerOptions.JOB_WORKERS_COUNT, "1",
-            ComputerOptions.JOB_PARTITIONS_COUNT, "1",
-            ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
-            ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "1000",
-            ComputerOptions.HGKV_MERGE_FILES_NUM, "5"
+                ComputerOptions.JOB_ID, "local_001",
+                ComputerOptions.JOB_WORKERS_COUNT, "1",
+                ComputerOptions.JOB_PARTITIONS_COUNT, "1",
+                ComputerOptions.WORKER_DATA_DIRS, "[data_dir1, data_dir2]",
+                ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "1000",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "5",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -142,7 +144,8 @@
                 ComputerOptions.WORKER_RECEIVED_BUFFERS_BYTES_LIMIT, "10000",
                 ComputerOptions.HGKV_MERGE_FILES_NUM, "5",
                 ComputerOptions.WORKER_VERTEX_PROPERTIES_COMBINER_CLASS,
-                MergeNewPropertiesCombiner.class.getName()
+                MergeNewPropertiesCombiner.class.getName(),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         FileUtils.deleteQuietly(new File("data_dir1"));
         FileUtils.deleteQuietly(new File("data_dir2"));
@@ -180,7 +183,7 @@
         Assert.assertFalse(it.hasNext());
     }
 
-    public static void addTenVertexBuffer(Consumer<ManagedBuffer> consumer)
+    public static void addTenVertexBuffer(Consumer<NetworkBuffer> consumer)
                                           throws IOException {
         for (long i = 0L; i < 10L; i++) {
             Vertex vertex = graphFactory().createVertex();
@@ -191,7 +194,7 @@
     }
 
     private static void addTwentyDuplicateVertexBuffer(
-                        Consumer<ManagedBuffer> consumer)
+                        Consumer<NetworkBuffer> consumer)
                         throws IOException {
         for (long i = 0L; i < 10L; i++) {
             Vertex vertex = graphFactory().createVertex();
@@ -214,7 +217,7 @@
         }
     }
 
-    private static void addTwoEmptyBuffer(Consumer<ManagedBuffer> consumer) {
+    private static void addTwoEmptyBuffer(Consumer<NetworkBuffer> consumer) {
         for (int i = 0; i < 2; i++) {
             ReceiverUtil.consumeBuffer(new byte[2], consumer);
         }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffersTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffersTest.java
index 065758c..201eacd 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffersTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sender/WriteBuffersTest.java
@@ -35,8 +35,8 @@
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.StreamGraphInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryInputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryInputImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.testutil.Whitebox;
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterTestUtil.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterTestUtil.java
index 2bb165e..146ec79 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterTestUtil.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/SorterTestUtil.java
@@ -30,6 +30,8 @@
 import com.baidu.hugegraph.computer.core.combiner.Combiner;
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
 import com.baidu.hugegraph.computer.core.common.Constants;
+import com.baidu.hugegraph.computer.core.config.ComputerOptions;
+import com.baidu.hugegraph.computer.core.config.Config;
 import com.baidu.hugegraph.computer.core.graph.id.Id;
 import com.baidu.hugegraph.computer.core.io.BytesInput;
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
@@ -37,8 +39,8 @@
 import com.baidu.hugegraph.computer.core.io.Readable;
 import com.baidu.hugegraph.computer.core.io.Writable;
 import com.baidu.hugegraph.computer.core.store.StoreTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
 import com.baidu.hugegraph.testutil.Assert;
 
 public class SorterTestUtil {
@@ -165,4 +167,12 @@
                                         Combiner<T> combiner) {
         return new AbstractPointerCombiner<T>(supplier, combiner) { };
     }
+
+    public static Sorter createSorter(Config config) {
+        if (config.get(ComputerOptions.TRANSPORT_RECV_FILE_MODE)) {
+            return new BufferFileSorter(config);
+        } else {
+            return new HgkvFileSorter(config);
+        }
+    }
 }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/FlusherTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/FlusherTest.java
index 2f987f1..1d8436a 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/FlusherTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/FlusherTest.java
@@ -23,12 +23,13 @@
 import java.util.Iterator;
 import java.util.List;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.baidu.hugegraph.computer.core.combiner.OverwriteCombiner;
 import com.baidu.hugegraph.computer.core.combiner.PointerCombiner;
-import com.baidu.hugegraph.computer.core.common.ComputerContext;
 import com.baidu.hugegraph.computer.core.common.Constants;
+import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
 import com.baidu.hugegraph.computer.core.graph.value.IntValue;
 import com.baidu.hugegraph.computer.core.io.BytesInput;
@@ -36,23 +37,30 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.sort.Sorter;
-import com.baidu.hugegraph.computer.core.sort.SorterImpl;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.KvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.KvOuterSortFlusher;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
 import com.baidu.hugegraph.computer.core.store.StoreTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.google.common.collect.ImmutableList;
 
 public class FlusherTest {
 
-    private static final Config CONFIG = ComputerContext.instance().config();
+    private static Config CONFIG;
+
+    @BeforeClass
+    public static void init() {
+        CONFIG = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
+        );
+    }
 
     @Test
     public void testKvInnerSortFlusher() throws Exception {
@@ -64,7 +72,7 @@
 
         BytesOutput output = IOFactory.createBytesOutput(
                              Constants.SMALL_BUF_SIZE);
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
         sorter.sortBuffer(input, new KvInnerSortFlusher(output), false);
 
         BytesInput result = EntriesUtil.inputFromOutput(output);
@@ -90,7 +98,7 @@
         List<RandomAccessInput> inputs = ImmutableList.of(input1, input2);
 
         String resultFile = StoreTestUtil.availablePathById("1");
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
         sorter.mergeBuffers(inputs, new KvOuterSortFlusher(), resultFile,
                             false);
 
@@ -136,7 +144,7 @@
                                                   new OverwriteCombiner<>());
         InnerSortFlusher flusher = new CombineKvInnerSortFlusher(output,
                                                                  combiner);
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
         sorter.sortBuffer(input, flusher, false);
 
         BytesInput result = EntriesUtil.inputFromOutput(output);
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SortLargeDataTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SortLargeDataTest.java
index 3635f31..ed375b0 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SortLargeDataTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SortLargeDataTest.java
@@ -44,7 +44,6 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.sort.Sorter;
-import com.baidu.hugegraph.computer.core.sort.SorterImpl;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvOuterSortFlusher;
@@ -52,16 +51,16 @@
 import com.baidu.hugegraph.computer.core.sort.flusher.KvOuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.PeekableIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileWriter;
 import com.baidu.hugegraph.computer.core.store.StoreTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.DefaultKvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.InlinePointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.entry.DefaultKvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.InlinePointer;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvDirBuilderImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.util.Bytes;
@@ -78,7 +77,8 @@
     public static void init() {
         CONFIG = UnitTestBase.updateWithRequiredOptions(
                 ComputerOptions.HGKV_MERGE_FILES_NUM, "200",
-                ComputerOptions.HGKV_MAX_FILE_SIZE, String.valueOf(Bytes.GB)
+                ComputerOptions.HGKV_MAX_FILE_SIZE, String.valueOf(Bytes.GB),
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
     }
 
@@ -106,7 +106,7 @@
         List<RandomAccessInput> buffers = new ArrayList<>(mergeBufferNum);
         List<String> mergeBufferFiles = new ArrayList<>();
         int fileNum = 10;
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
 
         watcher.start();
         for (int i = 0; i < dataSize; i++) {
@@ -168,7 +168,7 @@
         }
 
         // Sort buffer
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
         watcher.start();
         List<RandomAccessInput> sortedBuffers = new ArrayList<>();
         for (RandomAccessInput buffer : buffers) {
@@ -212,7 +212,7 @@
         }
 
         String resultFile = StoreTestUtil.availablePathById("0");
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(CONFIG);
         mergeBuffers(sorter, buffers, resultFile);
 
         // Assert result
@@ -223,7 +223,8 @@
     @Test
     public void testDiffNumEntriesFileMerge() throws Exception {
         Config config = UnitTestBase.updateWithRequiredOptions(
-                ComputerOptions.HGKV_MERGE_FILES_NUM, "3"
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "3",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         List<Integer> sizeList = ImmutableList.of(200, 500, 20, 50, 300,
                                                   250, 10, 33, 900, 89, 20);
@@ -232,8 +233,8 @@
         for (int j = 0; j < sizeList.size(); j++) {
             String file = StoreTestUtil.availablePathById(j + 10);
             inputs.add(file);
-            try (HgkvDirBuilder builder = new HgkvDirBuilderImpl(config,
-                                                                 file)) {
+            try (KvEntryFileWriter builder = new HgkvDirBuilderImpl(config,
+                                                                    file)) {
                 for (int i = 0; i < sizeList.get(j); i++) {
                     byte[] keyBytes = StoreTestUtil.intToByteArray(i);
                     byte[] valueBytes = StoreTestUtil.intToByteArray(1);
@@ -250,7 +251,7 @@
                                StoreTestUtil.availablePathById(1),
                                StoreTestUtil.availablePathById(2),
                                StoreTestUtil.availablePathById(3));
-        Sorter sorter = new SorterImpl(config);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         sorter.mergeInputs(inputs, new KvOuterSortFlusher(), outputs, false);
 
         int total = sizeList.stream().mapToInt(i -> i).sum();
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SorterTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SorterTest.java
index 6e51a48..fcd359c 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SorterTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/sort/sorter/SorterTest.java
@@ -27,7 +27,6 @@
 import org.apache.commons.io.FileUtils;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.baidu.hugegraph.computer.core.combiner.IntValueSumCombiner;
@@ -41,7 +40,6 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.sort.Sorter;
-import com.baidu.hugegraph.computer.core.sort.SorterImpl;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvInnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineKvOuterSortFlusher;
@@ -49,17 +47,18 @@
 import com.baidu.hugegraph.computer.core.sort.flusher.CombineSubKvOuterSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.InnerSortFlusher;
 import com.baidu.hugegraph.computer.core.sort.flusher.OuterSortFlusher;
+import com.baidu.hugegraph.computer.core.store.EntryIterator;
+import com.baidu.hugegraph.computer.core.store.KvEntryFileReader;
 import com.baidu.hugegraph.computer.core.store.StoreTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReader;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReaderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.select.DisperseEvenlySelector;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.select.InputFilesSelector;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader.HgkvDirReaderImpl;
+import com.baidu.hugegraph.computer.core.store.file.select.DisperseEvenlySelector;
+import com.baidu.hugegraph.computer.core.store.file.select.InputFilesSelector;
+import com.baidu.hugegraph.computer.core.util.FileUtil;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.iterator.CIter;
 import com.baidu.hugegraph.testutil.Assert;
@@ -68,17 +67,6 @@
 
 public class SorterTest {
 
-    private static Config CONFIG;
-
-    @BeforeClass
-    public static void init() {
-        CONFIG = UnitTestBase.updateWithRequiredOptions(
-                ComputerOptions.HGKV_MAX_FILE_SIZE, "32",
-                ComputerOptions.HGKV_DATABLOCK_SIZE, "16",
-                ComputerOptions.HGKV_MERGE_FILES_NUM, "3"
-        );
-    }
-
     @Before
     public void setup() throws IOException {
         FileUtils.deleteDirectory(new File(StoreTestUtil.FILE_DIR));
@@ -91,6 +79,11 @@
 
     @Test
     public void testSortKvBuffer() throws Exception {
+        Config config = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.HGKV_MAX_FILE_SIZE, "32",
+                ComputerOptions.HGKV_DATABLOCK_SIZE, "16",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "3"
+        );
         List<Integer> map = ImmutableList.of(2, 3,
                                              1, 23,
                                              6, 2,
@@ -102,7 +95,7 @@
         BytesOutput output = IOFactory.createBytesOutput(
                              Constants.SMALL_BUF_SIZE);
 
-        SorterImpl sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         PointerCombiner combiner = SorterTestUtil.createPointerCombiner(
                                                   IntValue::new,
                                                   new IntValueSumCombiner());
@@ -121,6 +114,12 @@
 
     @Test
     public void testSortKvBuffers() throws Exception {
+        Config config = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.HGKV_MAX_FILE_SIZE, "32",
+                ComputerOptions.HGKV_DATABLOCK_SIZE, "16",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "3",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
+        );
         List<Integer> map1 = ImmutableList.of(2, 3,
                                               2, 1,
                                               5, 2,
@@ -139,7 +138,7 @@
                                 SorterTestUtil.inputFromKvMap(map2),
                                 SorterTestUtil.inputFromKvMap(map1),
                                 SorterTestUtil.inputFromKvMap(map2));
-        SorterImpl sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         PointerCombiner combiner = SorterTestUtil.createPointerCombiner(
                                                   IntValue::new,
                                                   new IntValueSumCombiner());
@@ -147,7 +146,7 @@
                             path, false);
 
         // Assert merge result from target hgkvDir
-        HgkvDirReader reader = new HgkvDirReaderImpl(path, false);
+        KvEntryFileReader reader = new HgkvDirReaderImpl(path, false);
         EntryIterator iter = reader.iterator();
         SorterTestUtil.assertKvEntry(iter.next(), 1, 8);
         SorterTestUtil.assertKvEntry(iter.next(), 2, 8);
@@ -160,6 +159,24 @@
 
     @Test
     public void testMergeKvInputs() throws Exception {
+        Config config = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.HGKV_MAX_FILE_SIZE, "32",
+                ComputerOptions.HGKV_DATABLOCK_SIZE, "16",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "3",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
+        );
+        this.testMergeKvInputs(config);
+
+        config = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.HGKV_MAX_FILE_SIZE, "32",
+                ComputerOptions.HGKV_DATABLOCK_SIZE, "16",
+                ComputerOptions.HGKV_MERGE_FILES_NUM, "3",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "true"
+        );
+        this.testMergeKvInputs(config);
+    }
+
+    private void testMergeKvInputs(Config config) throws Exception {
         List<Integer> map1 = ImmutableList.of(2, 3,
                                               2, 1,
                                               5, 2,
@@ -198,11 +215,15 @@
             } else {
                 map = map2;
             }
-            StoreTestUtil.hgkvDirFromKvMap(CONFIG, map, inputs.get(i));
+            if (config.get(ComputerOptions.TRANSPORT_RECV_FILE_MODE)) {
+                StoreTestUtil.bufferFileFromKvMap(map, inputs.get(i));
+            } else {
+                StoreTestUtil.hgkvDirFromKvMap(config, map, inputs.get(i));
+            }
         }
 
         // Merge file
-        Sorter sorter = new SorterImpl(CONFIG);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         PointerCombiner combiner = SorterTestUtil.createPointerCombiner(
                                                   IntValue::new,
                                                   new IntValueSumCombiner());
@@ -241,6 +262,9 @@
             value = StoreTestUtil.dataFromPointer(last.value());
         }
         Assert.assertFalse(resultIter.hasNext());
+
+        FileUtil.deleteFilesQuietly(inputs);
+        FileUtil.deleteFilesQuietly(outputs);
     }
 
     private BytesInput sortedSubKvBuffer(Config config) throws Exception {
@@ -272,7 +296,7 @@
         InnerSortFlusher flusher = new CombineSubKvInnerSortFlusher(
                                        output, combiner, flushThreshold);
 
-        Sorter sorter = new SorterImpl(config);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         sorter.sortBuffer(input, flusher, true);
 
         return EntriesUtil.inputFromOutput(output);
@@ -305,7 +329,8 @@
     @Test
     public void testSortSubKvBuffers() throws Exception {
         Config config = UnitTestBase.updateWithRequiredOptions(
-            ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "2"
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "2",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
         int flushThreshold = config.get(
                              ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX);
@@ -315,7 +340,7 @@
         BytesInput i3 = this.sortedSubKvBuffer(config);
         List<RandomAccessInput> buffers = ImmutableList.of(i1, i2, i3);
 
-        Sorter sorter = new SorterImpl(config);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         PointerCombiner combiner = SorterTestUtil.createPointerCombiner(
                                                   IntValue::new,
                                                   new IntValueSumCombiner());
@@ -351,10 +376,21 @@
     @Test
     public void testMergeSubKvFiles() throws Exception {
         Config config = UnitTestBase.updateWithRequiredOptions(
-                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "2"
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "2",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "false"
         );
+        this.testMergeSubKvFiles(config);
+
+        config = UnitTestBase.updateWithRequiredOptions(
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX, "2",
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE, "true"
+        );
+        this.testMergeSubKvFiles(config);
+    }
+
+    private void testMergeSubKvFiles(Config config) throws Exception {
         int flushThreshold = config.get(
-                             ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX);
+                ComputerOptions.INPUT_MAX_EDGES_IN_ONE_VERTEX);
 
         List<Integer> kv1 = ImmutableList.of(1,
                                              2, 1,
@@ -378,6 +414,7 @@
         List<List<Integer>> data1 = ImmutableList.of(kv1, kv2, kv3);
         List<List<Integer>> data2 = ImmutableList.of(kv4, kv5, kv6);
         List<List<Integer>> data3 = ImmutableList.of(kv4, kv1, kv3);
+        List<List<List<Integer>>> datas = ImmutableList.of(data1, data2, data3);
 
         String input1 = StoreTestUtil.availablePathById(1);
         String input2 = StoreTestUtil.availablePathById(2);
@@ -387,16 +424,24 @@
         List<String> inputs = ImmutableList.of(input1, input2, input3);
         List<String> outputs = ImmutableList.of(output);
 
-        StoreTestUtil.hgkvDirFromSubKvMap(config, data1, input1);
-        StoreTestUtil.hgkvDirFromSubKvMap(config, data2, input2);
-        StoreTestUtil.hgkvDirFromSubKvMap(config, data3, input3);
+        boolean useBufferFile = config.get(
+                ComputerOptions.TRANSPORT_RECV_FILE_MODE);
+        for (int i = 0; i < inputs.size(); i++) {
+            String input = inputs.get(i);
+            List<List<Integer>> data = datas.get(i);
+            if (useBufferFile) {
+                StoreTestUtil.bufferFileFromSubKvMap(data, input);
+            } else {
+                StoreTestUtil.hgkvDirFromSubKvMap(config, data, input);
+            }
+        }
 
-        Sorter sorter = new SorterImpl(config);
+        Sorter sorter = SorterTestUtil.createSorter(config);
         PointerCombiner combiner = SorterTestUtil.createPointerCombiner(
-                                                  IntValue::new,
-                                                  new IntValueSumCombiner());
+                IntValue::new,
+                new IntValueSumCombiner());
         OuterSortFlusher flusher = new CombineSubKvOuterSortFlusher(
-                                       combiner, flushThreshold);
+                combiner, flushThreshold);
         flusher.sources(inputs.size());
         sorter.mergeInputs(inputs, flusher, outputs, true);
 
@@ -416,6 +461,9 @@
             SorterTestUtil.assertSubKvByKv(kvIter.next(), 4, 2, 1, 3, 1);
             SorterTestUtil.assertSubKvByKv(kvIter.next(), 4, 6, 2, 8, 2);
         }
+
+        FileUtil.deleteFilesQuietly(inputs);
+        FileUtil.deleteFilesQuietly(outputs);
     }
 
     @Test
@@ -433,7 +481,7 @@
         InputFilesSelector selector = new DisperseEvenlySelector();
 
         Assert.assertThrows(IllegalArgumentException.class, () -> {
-            selector.selectedOfOutputs(inputs, outputs);
+            selector.selectedByHgkvFile(inputs, outputs);
         }, e -> {
             String errorMsg = "inputs size of InputFilesSelector must be >= " +
                               "outputs size";
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/BitFileTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/BitFileTest.java
index 4cc8632..65492d1 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/BitFileTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/BitFileTest.java
@@ -31,10 +31,10 @@
 
 import com.baidu.hugegraph.computer.core.common.ComputerContext;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.store.seqfile.BitsFileReader;
-import com.baidu.hugegraph.computer.core.store.seqfile.BitsFileReaderImpl;
-import com.baidu.hugegraph.computer.core.store.seqfile.BitsFileWriter;
-import com.baidu.hugegraph.computer.core.store.seqfile.BitsFileWriterImpl;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.BitsFileReader;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.BitsFileReaderImpl;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.BitsFileWriter;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.BitsFileWriterImpl;
 import com.baidu.hugegraph.testutil.Assert;
 
 public class BitFileTest {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntriesUtilTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntriesUtilTest.java
index 2fee7f8..de61a9a 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntriesUtilTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntriesUtilTest.java
@@ -29,13 +29,12 @@
 import com.baidu.hugegraph.computer.core.io.BytesInput;
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.SubKvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntryWriter;
+import com.baidu.hugegraph.computer.core.store.buffer.SubKvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntryWriter;
 import com.baidu.hugegraph.testutil.Assert;
 
 public class EntriesUtilTest {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntryOutputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntryOutputTest.java
index 825b260..648ffa9 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntryOutputTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/EntryOutputTest.java
@@ -34,13 +34,12 @@
 import com.baidu.hugegraph.computer.core.io.IOFactory;
 import com.baidu.hugegraph.computer.core.io.Writable;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryOutputImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntryWriter;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryOutputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntryWriter;
 import com.google.common.collect.ImmutableList;
 
 public class EntryOutputTest {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvDirTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvDirTest.java
index 265079c..5b6f233 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvDirTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvDirTest.java
@@ -32,15 +32,12 @@
 
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDir;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFileImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReader;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvDirReaderImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDir;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFileImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvDirBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader.HgkvDirReaderImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.google.common.collect.ImmutableList;
@@ -78,7 +75,7 @@
         List<KvEntry> kvEntries = StoreTestUtil.kvEntriesFromMap(data);
 
         String path = StoreTestUtil.availablePathById("1");
-        try (HgkvDirBuilder builder = new HgkvDirBuilderImpl(CONFIG, path)) {
+        try (KvEntryFileWriter builder = new HgkvDirBuilderImpl(CONFIG, path)) {
             for (KvEntry entry : kvEntries) {
                 builder.write(entry);
             }
@@ -111,7 +108,7 @@
                                               6, 2);
         String path = StoreTestUtil.availablePathById("1");
         StoreTestUtil.hgkvDirFromKvMap(CONFIG, data, path);
-        HgkvDirReader reader = new HgkvDirReaderImpl(path, false);
+        KvEntryFileReader reader = new HgkvDirReaderImpl(path, false);
         try (EntryIterator iterator = reader.iterator()) {
             int i = 0;
             while (iterator.hasNext()) {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvFileTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvFileTest.java
index 51c4ccb..a3d0e81 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvFileTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/HgkvFileTest.java
@@ -32,14 +32,12 @@
 
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.EntryIterator;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFile;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvFileImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvFileBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvFileBuilderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvFileReader;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.reader.HgkvFileReaderImpl;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFile;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvFileImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvFileBuilder;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvFileBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.reader.HgkvFileReaderImpl;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.google.common.collect.ImmutableList;
@@ -135,7 +133,8 @@
         String filePath = StoreTestUtil.availablePathById("1");
         File file = StoreTestUtil.mapToHgkvFile(CONFIG, data, filePath);
 
-        HgkvFileReader reader = new HgkvFileReaderImpl(file.getPath(), false);
+        KvEntryFileReader reader = new HgkvFileReaderImpl(file.getPath(),
+                                                          false);
         try (EntryIterator iterator = reader.iterator()) {
             int index = 0;
             while (iterator.hasNext()) {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/PointerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/PointerTest.java
index d93adbe..2094261 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/PointerTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/PointerTest.java
@@ -28,9 +28,9 @@
 import com.baidu.hugegraph.computer.core.io.BytesInput;
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntriesUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.entry.EntriesUtil;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
 import com.baidu.hugegraph.computer.core.util.BytesUtil;
 
 public class PointerTest {
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/StoreTestUtil.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/StoreTestUtil.java
index c902ece..db331db 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/StoreTestUtil.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/StoreTestUtil.java
@@ -36,15 +36,15 @@
 import com.baidu.hugegraph.computer.core.io.BytesInput;
 import com.baidu.hugegraph.computer.core.io.BytesOutput;
 import com.baidu.hugegraph.computer.core.io.IOFactory;
+import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
 import com.baidu.hugegraph.computer.core.sort.SorterTestUtil;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.buffer.KvEntriesInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.KvEntry;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.Pointer;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.HgkvDirImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvDirBuilderImpl;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvFileBuilder;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.file.builder.HgkvFileBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.buffer.KvEntriesInput;
+import com.baidu.hugegraph.computer.core.store.entry.KvEntry;
+import com.baidu.hugegraph.computer.core.store.entry.Pointer;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.HgkvDirImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvDirBuilderImpl;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvFileBuilder;
+import com.baidu.hugegraph.computer.core.store.file.hgkvfile.builder.HgkvFileBuilderImpl;
 import com.baidu.hugegraph.testutil.Assert;
 
 public class StoreTestUtil {
@@ -90,7 +90,7 @@
     public static void hgkvDirFromKvMap(Config config, List<Integer> map,
                                         String path) throws IOException {
         File file = new File(path);
-        try (HgkvDirBuilder builder = new HgkvDirBuilderImpl(config, path)) {
+        try (KvEntryFileWriter builder = new HgkvDirBuilderImpl(config, path)) {
             List<KvEntry> entries = StoreTestUtil.kvEntriesFromMap(map);
             for (KvEntry entry : entries) {
                 builder.write(entry);
@@ -107,7 +107,7 @@
                                            String path) throws IOException {
         BytesInput input = SorterTestUtil.inputFromSubKvMap(map);
         KvEntriesInput iter = new KvEntriesInput(input, true);
-        try (HgkvDirBuilder builder = new HgkvDirBuilderImpl(config, path)) {
+        try (KvEntryFileWriter builder = new HgkvDirBuilderImpl(config, path)) {
             while (iter.hasNext()) {
                 builder.write(iter.next());
             }
@@ -115,6 +115,22 @@
         iter.close();
     }
 
+    public static void bufferFileFromKvMap(List<Integer> map, String path)
+                                           throws IOException {
+        RandomAccessOutput output = IOFactory.createFileOutput(new File(path));
+        BytesInput buffer = SorterTestUtil.inputFromKvMap(map);
+        output.write(buffer.readBytes((int) buffer.available()));
+        output.close();
+    }
+
+    public static void bufferFileFromSubKvMap(List<List<Integer>> map,
+                                              String path) throws IOException {
+        RandomAccessOutput output = IOFactory.createFileOutput(new File(path));
+        BytesInput buffer = SorterTestUtil.inputFromSubKvMap(map);
+        output.write(buffer.readBytes((int) buffer.available()));
+        output.close();
+    }
+
     public static File mapToHgkvFile(Config config, List<Integer> map,
                                      String path) throws IOException {
         File file = new File(path);
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/ValueFileTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/ValueFileTest.java
index 950c714..fdc0628 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/ValueFileTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/store/ValueFileTest.java
@@ -33,8 +33,8 @@
 import com.baidu.hugegraph.computer.core.config.Config;
 import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput;
 import com.baidu.hugegraph.computer.core.io.UnsafeBytesOutput;
-import com.baidu.hugegraph.computer.core.store.seqfile.ValueFileInput;
-import com.baidu.hugegraph.computer.core.store.seqfile.ValueFileOutput;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.ValueFileInput;
+import com.baidu.hugegraph.computer.core.store.file.seqfile.ValueFileOutput;
 import com.baidu.hugegraph.computer.suite.unit.UnitTestBase;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.testutil.Whitebox;
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/integrate/SenderIntegrateTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/integrate/SenderIntegrateTest.java
index f1f7509..f4e3d8e 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/integrate/SenderIntegrateTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/integrate/SenderIntegrateTest.java
@@ -27,6 +27,7 @@
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
 
 import com.baidu.hugegraph.computer.algorithm.centrality.pagerank.PageRankParams;
 import com.baidu.hugegraph.computer.core.common.exception.TransportException;
@@ -45,9 +46,12 @@
 import com.baidu.hugegraph.config.RpcOptions;
 import com.baidu.hugegraph.testutil.Assert;
 import com.baidu.hugegraph.testutil.Whitebox;
+import com.baidu.hugegraph.util.Log;
 
 public class SenderIntegrateTest {
 
+    public static final Logger LOG = Log.logger(SenderIntegrateTest.class);
+
     private static final Class<?> COMPUTATION = MockComputation.class;
 
     @BeforeClass
@@ -114,7 +118,7 @@
     @Test
     public void testMultiWorkers() throws InterruptedException {
         int workerCount = 3;
-        int partitionCount = 5;
+        int partitionCount = 3;
         Thread masterThread = new Thread(() -> {
             String[] args = OptionsBuilder.newInstance()
                                           .withJobId("local_003")
@@ -198,6 +202,7 @@
             try (MasterService service = initMaster(args)) {
                 service.execute();
             } catch (Exception e) {
+                LOG.error("Failed to execute master service", e);
                 Assert.fail(e.getMessage());
             }
         });
@@ -221,6 +226,7 @@
                 this.slowSendFunc(service);
                 service.execute();
             } catch (Exception e) {
+                LOG.error("Failed to execute worker service", e);
                 Assert.fail(e.getMessage());
             }
         });
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/unit/UnitTestBase.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/unit/UnitTestBase.java
index 2f1e7b5..70cdc2d 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/unit/UnitTestBase.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/suite/unit/UnitTestBase.java
@@ -45,8 +45,8 @@
 import com.baidu.hugegraph.computer.core.io.StreamGraphInput;
 import com.baidu.hugegraph.computer.core.io.StreamGraphOutput;
 import com.baidu.hugegraph.computer.core.io.Writable;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryInput;
-import com.baidu.hugegraph.computer.core.store.hgkvfile.entry.EntryInputImpl;
+import com.baidu.hugegraph.computer.core.store.entry.EntryInput;
+import com.baidu.hugegraph.computer.core.store.entry.EntryInputImpl;
 import com.baidu.hugegraph.computer.core.util.ComputerContextUtil;
 import com.baidu.hugegraph.computer.core.worker.MockComputationParams;
 import com.baidu.hugegraph.config.TypedOption;
diff --git a/pom.xml b/pom.xml
index 2198618..cbb1eba 100644
--- a/pom.xml
+++ b/pom.xml
@@ -42,8 +42,8 @@
         <top.level.dir>${project.basedir}/..</top.level.dir>
         <release.name>hugegraph-computer</release.name>
         <final.name>${release.name}-${project.version}</final.name>
-        <compiler.source>11</compiler.source>
-        <compiler.target>11</compiler.target>
+        <compiler.source>1.8</compiler.source>
+        <compiler.target>1.8</compiler.target>
         <shell-executable>bash</shell-executable>
         <etcd.version>0.5.4</etcd.version>
         <hadoop-version>3.1.2</hadoop-version>