You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2019/07/05 08:27:00 UTC

[incubator-iotdb] branch master updated: New Storage Engine (#217)

This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 3a16bd5  New Storage Engine (#217)
3a16bd5 is described below

commit 3a16bd50517b303281a5c4a082c441770a8a4fe2
Author: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
AuthorDate: Fri Jul 5 16:26:56 2019 +0800

    New Storage Engine (#217)
    
    *  New Storage Engine. BufferwriteProcessor, OverflowProcessor, FileNodeProcessor and FileNodeManager are replaced by UnsealedTsFileProcessor, UnsealedTsFileProcessor, StorageGroupProcessor and StorageGroupManager.
    
    * Async Flush Processor
    
    * QueryDataSource is modified to adapt the new storage engine.
    
    * Data dir is modified. TsFile.restore, FileNode.restore are discarded.
    
    * a new tsfile summary file: tsfile.resource.
    
    * Overflow file is isomorphic with Tsfile.
    
    * A new chunk structure: TVList
    
    * Each tsfile has a WAL file.
    
    * Add some config parameters: memTableNumber, chunkBufferPoolEnable.
    
    * System_dir is discarded.
---
 .gitignore                                         |    6 +-
 .mvn/wrapper/MavenWrapperDownloader.java           |    4 -
 .../UserGuideV0.7.0/4-Deployment and Management.md |  264 +--
 .../apache/iotdb/kafka/KafkaConsumerThread.java    |    8 +-
 .../java/org/apache/iotdb/kafka/KafkaProducer.java |    4 +-
 .../org/apache/iotdb/example/RocketMQConsumer.java |    6 +-
 .../org/apache/iotdb/example/RocketMQProducer.java |    4 +-
 .../apache/iotdb/tsfile/hadoop/TSFInputFormat.java |   27 +-
 .../iotdb/tsfile/hadoop/TSFOutputFormat.java       |    8 +-
 .../iotdb/tsfile/hadoop/TSFRecordReader.java       |   16 +-
 .../iotdb/tsfile/hadoop/TSFRecordWriter.java       |    4 +-
 .../iotdb/tsfile/hadoop/example/TsFileHelper.java  |    6 +-
 .../org/apache/iotdb/cli/tool/ImportCsvTestIT.java |    1 -
 iotdb/iotdb/conf/iotdb-engine.properties           |  146 +-
 iotdb/iotdb/conf/iotdb-env.sh                      |    4 +-
 iotdb/iotdb/conf/tsfile-format.properties          |   14 +-
 iotdb/pom.xml                                      |   26 +
 .../iotdb/db/auth/authorizer/BasicAuthorizer.java  |   77 +-
 .../iotdb/db/auth/authorizer/IAuthorizer.java      |   59 +-
 .../db/auth/authorizer/LocalFileAuthorizer.java    |    4 +-
 .../iotdb/db/auth/role/LocalFileRoleAccessor.java  |    6 +-
 .../iotdb/db/auth/user/BasicUserManager.java       |    2 +-
 .../iotdb/db/auth/user/LocalFileUserAccessor.java  |   18 +-
 .../IoTDBDefaultThreadExceptionHandler.java        |    4 +-
 .../org/apache/iotdb/db/concurrent/ThreadName.java |    1 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  595 +-----
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |    7 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  182 +-
 .../iotdb/db/conf/directories/Directories.java     |  110 --
 .../db/conf/directories/DirectoryManager.java      |  156 ++
 .../directories/strategy/DirectoryStrategy.java    |   12 +-
 .../MinFolderOccupiedSpaceFirstStrategy.java       |    2 +-
 .../java/org/apache/iotdb/db/engine/Processor.java |  191 --
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  356 ++++
 .../apache/iotdb/db/engine/bufferwrite/Action.java |   28 -
 .../db/engine/bufferwrite/ActionException.java     |   30 -
 .../engine/bufferwrite/BufferWriteProcessor.java   |  561 ------
 .../db/engine/bufferwrite/FileNodeConstants.java   |   44 -
 .../bufferwrite/RestorableTsFileIOWriter.java      |  326 ---
 ...MetaDataCache.java => DeviceMetaDataCache.java} |   24 +-
 .../iotdb/db/engine/cache/TsFileMetaDataCache.java |   10 +-
 .../iotdb/db/engine/cache/TsFileMetadataUtils.java |   16 +-
 .../db/engine/filenode/FileNodeFlushFuture.java    |   91 -
 .../iotdb/db/engine/filenode/FileNodeManager.java  | 1252 ------------
 .../db/engine/filenode/FileNodeProcessor.java      | 2066 --------------------
 .../engine/filenode/FileNodeProcessorStatus.java   |   51 -
 .../db/engine/filenode/FileNodeProcessorStore.java |  165 --
 .../db/engine/filenode/OverflowChangeType.java     |   57 -
 .../iotdb/db/engine/filenode/TsFileResource.java   |  390 ----
 .../db/engine/memcontrol/BasicMemController.java   |  186 --
 .../engine/memcontrol/DisabledMemController.java   |   56 -
 .../db/engine/memcontrol/FlushPartialPolicy.java   |   72 -
 .../db/engine/memcontrol/ForceFLushAllPolicy.java  |   61 -
 .../db/engine/memcontrol/JVMMemController.java     |  100 -
 .../db/engine/memcontrol/MemMonitorThread.java     |   88 -
 .../db/engine/memcontrol/MemStatisticThread.java   |  102 -
 .../iotdb/db/engine/memcontrol/NoActPolicy.java    |   38 -
 .../apache/iotdb/db/engine/memcontrol/Policy.java  |   28 -
 .../db/engine/memcontrol/RecordMemController.java  |  177 --
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  106 +-
 .../iotdb/db/engine/memtable/ChunkBufferPool.java  |  117 ++
 .../apache/iotdb/db/engine/memtable/IMemTable.java |   42 +-
 .../db/engine/memtable/IWritableMemChunk.java      |   26 +-
 .../db/engine/memtable/MemSeriesLazyMerger.java    |   11 -
 .../db/engine/memtable/MemTableFlushTask.java      |  244 +++
 .../db/engine/memtable/MemTableFlushUtil.java      |  110 --
 ...itiveMemTable.java => NotifyFlushMemTable.java} |   26 +-
 .../db/engine/memtable/PrimitiveMemTable.java      |   17 +-
 .../db/engine/memtable/TimeValuePairSorter.java    |    3 +-
 .../iotdb/db/engine/memtable/WritableMemChunk.java |  118 +-
 .../iotdb/db/engine/modification/Deletion.java     |    3 +-
 .../iotdb/db/engine/modification/Modification.java |   21 +-
 .../db/engine/modification/ModificationFile.java   |    6 +-
 .../io/LocalTextModificationAccessor.java          |   37 +-
 .../engine/modification/io/ModificationReader.java |    2 +-
 .../iotdb/db/engine/overflow/io/OverflowIO.java    |  173 --
 .../db/engine/overflow/io/OverflowMemtable.java    |  116 --
 .../db/engine/overflow/io/OverflowProcessor.java   |  737 -------
 .../db/engine/overflow/io/OverflowResource.java    |  329 ----
 .../overflow/io/OverflowedTsFileIOWriter.java      |   36 -
 .../engine/overflow/metadata/OFFileMetadata.java   |  107 -
 .../overflow/metadata/OFRowGroupListMetadata.java  |  109 --
 .../overflow/metadata/OFSeriesListMetadata.java    |  107 -
 .../db/engine/overflow/utils/MergeStatus.java      |   29 -
 .../db/engine/overflow/utils/OverflowOpType.java   |   29 -
 .../{FlushManager.java => FlushPoolManager.java}   |   61 +-
 .../db/engine/pool/FlushSubTaskPoolManager.java    |   95 +
 .../apache/iotdb/db/engine/pool/MergeManager.java  |  126 --
 .../querycontext/GlobalSortedSeriesDataSource.java |   92 -
 .../engine/querycontext/MergeSeriesDataSource.java |   33 -
 .../db/engine/querycontext/OverflowInsertFile.java |   51 -
 .../querycontext/OverflowSeriesDataSource.java     |   78 -
 .../querycontext/OverflowUpdateDeleteFile.java     |   42 -
 .../db/engine/querycontext/QueryDataSource.java    |   33 +-
 .../db/engine/querycontext/ReadOnlyMemChunk.java   |   54 +-
 .../db/engine/querycontext/UnsealedTsFile.java     |   44 -
 .../iotdb/db/engine/storagegroup/FlushManager.java |   76 +
 .../engine/storagegroup/StorageGroupProcessor.java |  645 ++++++
 .../db/engine/storagegroup/TsFileProcessor.java    |  532 +++++
 .../db/engine/storagegroup/TsFileResource.java     |  227 +++
 .../version/SimpleFileVersionController.java       |    6 +-
 .../exception/DiskSpaceInsufficientException.java  |    2 +-
 .../db/exception/FileNodeNotExistException.java    |   33 -
 ...erException.java => FlushRunTimeException.java} |   14 +-
 .../iotdb/db/exception/NotConsistentException.java |   29 -
 .../db/exception/OverflowProcessorException.java   |   41 -
 .../exception/OverflowWrongParameterException.java |   42 -
 ...ception.java => QueryInBatchStmtException.java} |   21 +-
 ...rException.java => StorageEngineException.java} |   10 +-
 ...ion.java => StorageEngineFailureException.java} |   14 +-
 ...on.java => StorageGroupProcessorException.java} |   12 +-
 ...xception.java => TsFileProcessorException.java} |   15 +-
 .../db/exception/builder/ExceptionBuilder.java     |    8 +-
 .../codebased/InvalidParameterException.java       |    4 +-
 .../java/org/apache/iotdb/db/metadata/MGraph.java  |   97 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  420 ++--
 .../java/org/apache/iotdb/db/metadata/MNode.java   |    6 +-
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  198 +-
 .../apache/iotdb/db/metadata/MetadataConstant.java |    3 -
 .../apache/iotdb/db/monitor/MonitorConstants.java  |   10 +-
 .../org/apache/iotdb/db/monitor/StatMonitor.java   |   69 +-
 .../iotdb/db/monitor/collector/FileSize.java       |   61 +-
 .../monitor/collector/MemTableWriteTimeCost.java   |   85 +
 .../org/apache/iotdb/db/qp/QueryProcessor.java     |    6 +-
 .../apache/iotdb/db/qp/constant/DatetimeUtils.java |    8 +-
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   13 +-
 .../iotdb/db/qp/constant/TSParserConstant.java     |   12 +-
 ...utor.java => AbstractQueryProcessExecutor.java} |   33 +-
 .../db/qp/executor/IQueryProcessExecutor.java      |   34 +-
 .../iotdb/db/qp/executor/OverflowQPExecutor.java   |  649 ------
 .../iotdb/db/qp/executor/QueryProcessExecutor.java |  622 +++++-
 .../db/qp/logical/crud/BasicFunctionOperator.java  |   24 +-
 .../iotdb/db/qp/logical/crud/FilterOperator.java   |   15 +-
 .../iotdb/db/qp/logical/crud/FromOperator.java     |    3 +-
 .../iotdb/db/qp/logical/crud/FunctionOperator.java |    4 +-
 .../iotdb/db/qp/logical/crud/InsertOperator.java   |    2 -
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |   81 +-
 .../iotdb/db/qp/physical/crud/AggregationPlan.java |    1 -
 .../iotdb/db/qp/physical/crud/DeletePlan.java      |   21 +-
 .../iotdb/db/qp/physical/crud/FillQueryPlan.java   |    1 -
 .../iotdb/db/qp/physical/crud/GroupByPlan.java     |    1 -
 .../iotdb/db/qp/physical/crud/InsertPlan.java      |  100 +-
 .../iotdb/db/qp/physical/crud/QueryPlan.java       |    1 -
 .../iotdb/db/qp/physical/crud/UpdatePlan.java      |   17 +-
 .../iotdb/db/qp/physical/sys/AuthorPlan.java       |    1 -
 .../iotdb/db/qp/physical/sys/LoadDataPlan.java     |    1 -
 .../iotdb/db/qp/physical/sys/MetadataPlan.java     |    4 +-
 .../iotdb/db/qp/physical/sys/PropertyPlan.java     |    1 -
 .../iotdb/db/qp/physical/transfer/Codec.java       |   29 -
 .../db/qp/physical/transfer/CodecInstances.java    |  472 -----
 .../db/qp/physical/transfer/PhysicalPlanCodec.java |   57 -
 .../physical/transfer/PhysicalPlanLogTransfer.java |   90 -
 .../db/qp/physical/transfer/SystemLogOperator.java |   35 -
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  163 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   24 +-
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |    2 -
 .../optimizer/MergeSingleFilterOptimizer.java      |   34 +-
 .../db/query/aggregation/AggregateFunction.java    |    4 +-
 .../db/query/aggregation/impl/CountAggrFunc.java   |   10 +-
 .../db/query/aggregation/impl/FirstAggrFunc.java   |    4 +-
 .../db/query/aggregation/impl/LastAggrFunc.java    |    4 +-
 .../db/query/aggregation/impl/MaxTimeAggrFunc.java |    6 +-
 .../query/aggregation/impl/MaxValueAggrFunc.java   |    4 +-
 .../db/query/aggregation/impl/MeanAggrFunc.java    |    4 +-
 .../db/query/aggregation/impl/MinTimeAggrFunc.java |    4 +-
 .../query/aggregation/impl/MinValueAggrFunc.java   |    4 +-
 .../iotdb/db/query/context/QueryContext.java       |    9 +-
 .../iotdb/db/query/control/FileReaderManager.java  |   15 +-
 .../iotdb/db/query/control/JobFileManager.java     |   24 +-
 .../db/query/control/QueryResourceManager.java     |   64 +-
 .../apache/iotdb/db/query/dataset/AuthDataSet.java |    6 +-
 ...ator.java => EngineDataSetWithValueFilter.java} |   26 +-
 ...r.java => EngineDataSetWithoutValueFilter.java} |   28 +-
 .../groupby/GroupByWithValueFilterDataSet.java     |   20 +-
 ....java => GroupByWithoutValueFilterDataSet.java} |   44 +-
 .../AbstractExecutorWithoutTimeGenerator.java      |   84 -
 .../db/query/executor/AggregateEngineExecutor.java |   67 +-
 .../iotdb/db/query/executor/EngineExecutor.java    |  127 ++
 .../executor/EngineExecutorWithTimeGenerator.java  |   89 -
 .../EngineExecutorWithoutTimeGenerator.java        |   77 -
 .../iotdb/db/query/executor/EngineQueryRouter.java |   45 +-
 .../db/query/executor/FillEngineExecutor.java      |   15 +-
 .../db/query/executor/IEngineQueryRouter.java      |   10 +-
 .../db/query/factory/ISeriesReaderFactory.java     |   90 +
 .../db/query/factory/SeriesReaderFactory.java      |  275 ---
 .../db/query/factory/SeriesReaderFactoryImpl.java  |  286 +++
 .../java/org/apache/iotdb/db/query/fill/IFill.java |   26 +-
 .../org/apache/iotdb/db/query/fill/LinearFill.java |    8 +-
 .../apache/iotdb/db/query/fill/PreviousFill.java   |    8 +-
 .../apache/iotdb/db/query/reader/IPointReader.java |    8 +-
 .../org/apache/iotdb/db/query/reader/IReader.java  |   36 -
 ...derByTimeStamp.java => IReaderByTimeStamp.java} |    7 +-
 .../query/reader/SeriesReaderWithValueFilter.java  |   75 +
 ...er.java => SeriesReaderWithoutValueFilter.java} |   43 +-
 .../iotdb/db/query/reader/mem/MemChunkReader.java  |    6 +-
 .../reader/mem/MemChunkReaderByTimestamp.java      |    4 +-
 ...Timestamp.java => SeriesReaderByTimestamp.java} |   15 +-
 .../db/query/reader/sequence/IterateReader.java    |   84 +
 .../query/reader/sequence/SequenceDataReader.java  |  148 --
 .../sequence/SequenceDataReaderByTimestamp.java    |  108 -
 ...sFilesReader.java => SequenceSeriesReader.java} |  142 +-
 ...p.java => SequenceSeriesReaderByTimestamp.java} |   78 +-
 .../reader/sequence/UnSealedTsFileReader.java      |   65 +-
 .../sequence/UnSealedTsFileReaderByTimestamp.java  |   93 +
 .../sequence/UnSealedTsFilesReaderByTimestamp.java |   64 -
 .../sequence/adapter/FileSeriesReaderAdapter.java} |   49 +-
 .../adapter/SeriesReaderByTimestampAdapter.java}   |   35 +-
 ...EngineChunkReader.java => DiskChunkReader.java} |   21 +-
 ...estamp.java => DiskChunkReaderByTimestamp.java} |   14 +-
 .../UnsequenceSeriesReader.java}                   |   26 +-
 .../timegenerator/AbstractNodeConstructor.java     |   44 +-
 .../db/query/timegenerator/EngineLeafNode.java     |    6 +-
 .../query/timegenerator/EngineNodeConstructor.java |   22 +-
 .../query/timegenerator/EngineTimeGenerator.java   |    6 +-
 .../org/apache/iotdb/db/rescon/MemTablePool.java   |  105 +
 .../apache/iotdb/db/rescon/PrimitiveArrayPool.java |  162 ++
 .../apache/iotdb/db/rescon/TVListAllocator.java    |  113 ++
 .../TVListAllocatorMBean.java}                     |    9 +-
 .../apache/iotdb/db/service/CloseMergeService.java |  211 --
 .../java/org/apache/iotdb/db/service/IService.java |    2 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   91 +-
 .../org/apache/iotdb/db/service/IoTDBMBean.java    |    4 +-
 .../apache/iotdb/db/service/IoTDBShutdownHook.java |    9 +-
 .../org/apache/iotdb/db/service/JDBCService.java   |   45 +-
 .../iotdb/db/service/JDBCServiceEventHandler.java  |    4 +-
 .../org/apache/iotdb/db/service/JMXService.java    |   16 +-
 .../java/org/apache/iotdb/db/service/Monitor.java  |   47 +-
 .../org/apache/iotdb/db/service/MonitorMBean.java  |    8 -
 .../apache/iotdb/db/service/RegisterManager.java   |    8 +-
 .../org/apache/iotdb/db/service/ServiceType.java   |    5 +-
 .../org/apache/iotdb/db/service/StartupChecks.java |   59 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  523 +++--
 .../org/apache/iotdb/db/sql/parse/AstNode.java     |   82 +-
 .../org/apache/iotdb/db/sql/parse/ParseDriver.java |   42 +-
 .../iotdb/db/sync/conf/SyncSenderConfig.java       |   28 +-
 .../iotdb/db/sync/conf/SyncSenderDescriptor.java   |   26 +-
 ...cServiceManager.java => SyncServerManager.java} |   77 +-
 .../db/sync/receiver/SyncServiceEventHandler.java  |   55 -
 .../iotdb/db/sync/receiver/SyncServiceImpl.java    |   68 +-
 .../iotdb/db/sync/sender/SyncFileManager.java      |   44 +-
 .../iotdb/db/sync/sender/SyncSenderImpl.java       |   50 +-
 .../java/org/apache/iotdb/db/tools/WalChecker.java |   76 +-
 .../java/org/apache/iotdb/db/utils/AuthUtils.java  |   12 +-
 .../org/apache/iotdb/db/utils/CommonUtils.java     |   32 -
 .../iotdb/db/utils/CopyOnReadLinkedList.java       |   73 +
 .../org/apache/iotdb/db/utils/FilePathUtils.java   |    4 +
 .../org/apache/iotdb/db/utils/FileSchemaUtils.java |   12 +-
 .../java/org/apache/iotdb/db/utils/FileUtils.java  |   87 -
 .../java/org/apache/iotdb/db/utils/IOUtils.java    |    6 +-
 .../org/apache/iotdb/db/utils/ImmediateFuture.java |   58 -
 .../org/apache/iotdb/db/utils/LoadDataUtils.java   |  246 ---
 .../java/org/apache/iotdb/db/utils/MemUtils.java   |   76 +-
 .../org/apache/iotdb/db/utils/OpenFileNumUtil.java |   72 +-
 .../apache/iotdb/db/utils/PrimitiveArrayList.java  |  135 --
 .../iotdb/db/utils/PrimitiveArrayListFactory.java  |   47 -
 .../Utils.java => utils/QueryDataSetUtils.java}    |   14 +-
 .../org/apache/iotdb/db/utils/RecordUtils.java     |  115 --
 .../org/apache/iotdb/db/utils/TsPrimitiveType.java |   12 +-
 .../iotdb/db/utils/datastructure/BinaryTVList.java |  176 ++
 .../db/utils/datastructure/BooleanTVList.java      |  175 ++
 .../iotdb/db/utils/datastructure/DoubleTVList.java |  175 ++
 .../iotdb/db/utils/datastructure/FloatTVList.java  |  175 ++
 .../iotdb/db/utils/datastructure/IntTVList.java    |  175 ++
 .../iotdb/db/utils/datastructure/LongTVList.java   |  175 ++
 .../iotdb/db/utils/datastructure/TVList.java       |  396 ++++
 .../org/apache/iotdb/db/writelog/RecoverStage.java |   50 -
 .../iotdb/db/writelog/io/BatchLogReader.java       |   81 +
 .../apache/iotdb/db/writelog/io/ILogReader.java    |   25 +-
 .../apache/iotdb/db/writelog/io/ILogWriter.java    |   24 +-
 .../org/apache/iotdb/db/writelog/io/LogWriter.java |   66 +-
 .../iotdb/db/writelog/io/MultiFileLogReader.java   |   75 +
 .../apache/iotdb/db/writelog/io/RAFLogReader.java  |  104 -
 .../iotdb/db/writelog/io/SingleFileLogReader.java  |  133 ++
 .../writelog/manager/MultiFileLogNodeManager.java  |  114 +-
 .../db/writelog/manager/WriteLogNodeManager.java   |   30 +-
 .../db/writelog/node/ExclusiveWriteLogNode.java    |  282 ++-
 .../iotdb/db/writelog/node/WriteLogNode.java       |   49 +-
 .../recover/ExclusiveLogRecoverPerformer.java      |  366 ----
 .../writelog/recover/FileNodeRecoverPerformer.java |   55 -
 .../iotdb/db/writelog/recover/LogReplayer.java     |  146 ++
 .../db/writelog/recover/RecoverPerformer.java      |   32 -
 .../writelog/recover/TsFileRecoverPerformer.java   |  140 ++
 .../db/writelog/replay/ConcreteLogReplayer.java    |  102 -
 .../iotdb/db/writelog/replay/LogReplayer.java      |   27 -
 .../iotdb/db/auth/LocalFileAuthorizerTest.java     |   95 +-
 .../java/org/apache/iotdb/db/engine/PathUtils.java |   64 -
 .../org/apache/iotdb/db/engine/ProcessorTest.java  |  152 --
 .../engine/bufferwrite/BufferWriteBenchmark.java   |  127 --
 .../bufferwrite/BufferWriteProcessorNewTest.java   |  172 --
 .../bufferwrite/BufferWriteProcessorTest.java      |  257 ---
 .../bufferwrite/RestorableTsFileIOWriterTest.java  |  266 ---
 .../filenode/FileNodeProcessorStoreTest.java       |   91 -
 .../db/engine/filenode/TsFileResourceTest.java     |   98 -
 .../memcontrol/BufferwriteFileSizeControlTest.java |  164 --
 .../memcontrol/BufferwriteMetaSizeControlTest.java |  165 --
 .../db/engine/memcontrol/IoTDBMemControlTest.java  |  195 --
 .../db/engine/memcontrol/MemControllerTest.java    |   95 -
 .../memcontrol/OverflowFileSizeControlTest.java    |  143 --
 .../memcontrol/OverflowMetaSizeControlTest.java    |  144 --
 .../db/engine/memtable/ChunkBufferPoolTest.java    |   84 +
 .../db/engine/memtable/MemTableFlushTaskTest.java  |   81 +
 .../iotdb/db/engine/memtable/MemTablePoolTest.java |   95 +
 .../db/engine/memtable/PrimitiveMemTableTest.java  |    6 +-
 .../engine/modification/DeletionFileNodeTest.java  |  107 +-
 .../db/engine/modification/DeletionQueryTest.java  |  106 +-
 .../engine/modification/ModificationFileTest.java  |   17 +-
 .../io/LocalTextModificationAccessorTest.java      |    9 +-
 .../db/engine/overflow/io/OverflowIOTest.java      |   65 -
 .../engine/overflow/io/OverflowMemtableTest.java   |  100 -
 .../overflow/io/OverflowProcessorBenchmark.java    |  123 --
 .../engine/overflow/io/OverflowProcessorTest.java  |  211 --
 .../engine/overflow/io/OverflowResourceTest.java   |   92 -
 .../db/engine/overflow/io/OverflowTestUtils.java   |   77 -
 .../overflow/metadata/OFFileMetadataTest.java      |   90 -
 .../metadata/OFRowGroupListMetadataTest.java       |   93 -
 .../metadata/OFSeriesListMetadataTest.java         |   88 -
 .../overflow/metadata/OverflowTestHelper.java      |   84 -
 .../db/engine/overflow/metadata/OverflowUtils.java |  138 --
 .../FileNodeManagerBenchmark.java                  |   18 +-
 .../storagegroup/StorageGroupProcessorTest.java    |  110 ++
 .../engine/storagegroup/TsFileProcessorTest.java   |  208 ++
 .../apache/iotdb/db/integration/IOTDBFillIT.java   |    1 -
 .../iotdb/db/integration/IOTDBGroupByIT.java       |    1 -
 .../iotdb/db/integration/IoTDBAggregationIT.java   |    3 +-
 .../integration/IoTDBAggregationLargeDataIT.java   |    3 +-
 .../integration/IoTDBAggregationSmallDataIT.java   |    1 -
 .../iotdb/db/integration/IoTDBAuthorizationIT.java |    1 -
 .../iotdb/db/integration/IoTDBCompleteIT.java      |   35 +-
 .../apache/iotdb/db/integration/IoTDBDaemonIT.java |    1 -
 .../iotdb/db/integration/IoTDBDeletionIT.java      |   10 +-
 .../db/integration/IoTDBEngineTimeGeneratorIT.java |   11 +-
 .../db/integration/IoTDBFloatPrecisionIT.java      |    1 -
 .../db/integration/IoTDBFlushQueryMergeTest.java   |   11 +-
 .../iotdb/db/integration/IoTDBLargeDataIT.java     |    6 +-
 .../iotdb/db/integration/IoTDBLimitSlimitIT.java   |    1 -
 .../iotdb/db/integration/IoTDBMetadataFetchIT.java |    1 -
 .../iotdb/db/integration/IoTDBMultiSeriesIT.java   |    6 +-
 .../iotdb/db/integration/IoTDBPreparedStmtIT.java  |  223 +++
 .../db/integration/IoTDBSequenceDataQueryIT.java   |   10 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |   24 +-
 .../iotdb/db/integration/IoTDBTimeZoneIT.java      |    1 -
 .../iotdb/db/integration/IoTDBVersionIT.java       |    7 +-
 .../iotdb/db/metadata/MManagerAdvancedTest.java    |   13 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       |   95 +-
 .../iotdb/db/metadata/MManagerEfficiencyTest.java  |  155 --
 .../iotdb/db/metadata/MManagerImproveTest.java     |    5 +-
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |   53 +-
 .../org/apache/iotdb/db/monitor/MonitorTest.java   |  147 --
 .../iotdb/db/monitor/collector/FileSizeTest.java   |   10 +-
 .../org/apache/iotdb/db/qp/QueryProcessorTest.java |    4 +-
 .../transfer/PhysicalPlanLogTransferTest.java      |  162 --
 .../iotdb/db/qp/plan/LogicalPlanSmallTest.java     |   15 +-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |   13 +-
 .../org/apache/iotdb/db/qp/plan/QPUpdateTest.java  |   10 +-
 .../apache/iotdb/db/qp/utils/MemIntQpExecutor.java |   47 +-
 ....java => EngineDataSetWithValueFilterTest.java} |    6 +-
 .../query/reader/FakedSeriesReaderByTimestamp.java |    4 +-
 .../iotdb/db/query/reader/ReaderTestHelper.java    |   63 +
 ...ava => SeriesReaderWithoutValueFilterTest.java} |   10 +-
 .../reader/merge/SeriesMergeSortReaderTest.java    |    3 +-
 ...pTest.java => SeriesReaderByTimestampTest.java} |    7 +-
 ...erTest.java => UnsequenceSeriesReaderTest.java} |   17 +-
 .../query/reader/sequence/SeqDataReaderTest.java   |   98 +
 .../SequenceDataReaderByTimestampTest.java         |   91 -
 .../reader/sequence/UnsealedSeqReaderTest.java     |   93 +
 .../UnseqSeriesReaderByTimestampTest.java          |   97 +
 .../unsequence/UnsequenceSeriesReaderTest.java     |   95 +
 ...est.java => DatetimeQueryDataSetUtilsTest.java} |    2 +-
 .../db/sync/sender/MultipleClientSyncTest.java     |   18 +-
 .../iotdb/db/sync/sender/SingleClientSyncTest.java |   18 +-
 .../iotdb/db/sync/sender/SyncFileManagerTest.java  |   24 +-
 .../org/apache/iotdb/db/sync/test/RandomNum.java   |    2 +-
 .../apache/iotdb/db/sync/test/SyncTestClient2.java |   12 +-
 .../apache/iotdb/db/sync/test/SyncTestClient3.java |   12 +-
 .../org/apache/iotdb/db/tools/WalCheckerTest.java  |   19 +-
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  124 +-
 ...ilsTest.java => MathQueryDataSetUtilsTest.java} |    2 +-
 .../apache/iotdb/db/utils/OpenFileNumUtilTest.java |   70 +-
 .../iotdb/db/utils/PrimitiveArrayListTest.java     |   53 -
 .../db/utils/datastructure/LongTVListTest.java     |   93 +
 .../iotdb/db/writelog/IoTDBLogFileSizeTest.java    |   63 +-
 .../apache/iotdb/db/writelog/PerformanceTest.java  |   83 +-
 .../org/apache/iotdb/db/writelog/RecoverTest.java  |  310 ---
 .../iotdb/db/writelog/WriteLogNodeManagerTest.java |   79 +-
 .../apache/iotdb/db/writelog/WriteLogNodeTest.java |  182 +-
 .../iotdb/db/writelog/io/LogWriterReaderTest.java  |   33 +-
 .../db/writelog/io/MultiFileLogReaderTest.java     |   82 +
 .../iotdb/db/writelog/recover/LogReplayerTest.java |  124 ++
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |  149 ++
 .../writelog/recover/UnseqTsFileRecoverTest.java   |  160 ++
 .../org/apache/iotdb/jdbc/IoTDBConnection.java     |    5 +-
 .../apache/iotdb/jdbc/IoTDBDatabaseMetadata.java   |   14 +-
 .../java/org/apache/iotdb/jdbc/IoTDBDriver.java    |    4 +-
 .../jdbc/IoTDBPreparedInsertionStatement.java      |   73 +
 ...eStatement.java => IoTDBPreparedStatement.java} |   12 +-
 .../org/apache/iotdb/jdbc/IoTDBQueryResultSet.java |    4 +-
 .../java/org/apache/iotdb/jdbc/IoTDBStatement.java |   14 +-
 ...ntTest.java => IoTDBPreparedStatementTest.java} |   32 +-
 service-rpc/src/main/thrift/rpc.thrift             |   13 +
 service-rpc/src/main/thrift/sync.thrift            |    2 +-
 .../apache/iotdb/tsfile/TsFileSequenceRead.java    |    2 +-
 .../iotdb/tsfile/common/conf/TSFileConfig.java     |    9 +-
 .../iotdb/tsfile/common/conf/TSFileDescriptor.java |   26 +-
 .../tsfile/common/constant/SystemConstant.java     |    1 +
 .../iotdb/tsfile/compress/IUnCompressor.java       |    6 +-
 .../tsfile/encoding/decoder/BitmapDecoder.java     |    8 +-
 .../encoding/decoder/DoublePrecisionDecoder.java   |    6 +-
 .../tsfile/encoding/decoder/FloatDecoder.java      |   10 +-
 .../tsfile/encoding/decoder/GorillaDecoder.java    |    4 +-
 .../tsfile/encoding/decoder/IntRleDecoder.java     |    4 +-
 .../tsfile/encoding/decoder/LongRleDecoder.java    |    4 +-
 .../tsfile/encoding/decoder/PlainDecoder.java      |    6 +-
 .../encoding/decoder/SinglePrecisionDecoder.java   |    6 +-
 .../tsfile/encoding/encoder/BitmapEncoder.java     |    6 +-
 .../encoding/encoder/DeltaBinaryEncoder.java       |    4 +-
 .../encoding/encoder/DoublePrecisionEncoder.java   |    8 +-
 .../iotdb/tsfile/encoding/encoder/Encoder.java     |   16 +-
 .../tsfile/encoding/encoder/FloatEncoder.java      |    6 +-
 .../tsfile/encoding/encoder/PlainEncoder.java      |   17 +-
 .../iotdb/tsfile/encoding/encoder/RleEncoder.java  |   10 +-
 .../encoding/encoder/SinglePrecisionEncoder.java   |    2 +-
 .../tsfile/encoding/encoder/TSEncodingBuilder.java |    8 +-
 .../iotdb/tsfile/file/metadata/TsFileMetaData.java |    2 +-
 .../file/metadata/statistics/Statistics.java       |    2 +-
 .../iotdb/tsfile/read/TsFileRestorableReader.java  |   10 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |   82 +-
 .../apache/iotdb/tsfile/read/common/BatchData.java |    4 +-
 .../query/dataset/DataSetWithTimeGenerator.java    |   16 +-
 .../query/dataset/DataSetWithoutTimeGenerator.java |    2 +-
 .../query/executor/ExecutorWithTimeGenerator.java  |    6 +-
 .../tsfile/read/reader/DefaultTsFileInput.java     |    7 +-
 .../tsfile/read/reader/chunk/ChunkReader.java      |    2 +-
 ...stamp.java => FileSeriesReaderByTimestamp.java} |    4 +-
 .../tsfile/utils/ReadWriteForEncodingUtils.java    |    4 +-
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |   36 +-
 .../apache/iotdb/tsfile/write/TsFileWriter.java    |   38 +-
 .../iotdb/tsfile/write/chunk/ChunkBuffer.java      |   22 +-
 .../iotdb/tsfile/write/chunk/ChunkWriterImpl.java  |   18 +-
 .../iotdb/tsfile/write/chunk/IChunkWriter.java     |   14 +-
 .../apache/iotdb/tsfile/write/page/PageWriter.java |   16 +-
 .../tsfile/write/schema/MeasurementSchema.java     |    4 +-
 .../write/writer/IncompleteFileTestUtil.java       |    2 +-
 .../write/writer/NativeRestorableIOWriter.java     |   93 -
 .../write/writer/RestorableTsFileIOWriter.java     |  182 ++
 .../iotdb/tsfile/write/writer/TsFileIOWriter.java  |   49 +-
 .../tsfile/encoding/decoder/BitmapDecoderTest.java |    4 +-
 .../tsfile/encoding/decoder/FloatDecoderTest.java  |   16 +-
 .../encoding/decoder/GorillaDecoderTest.java       |    6 +-
 .../decoder/delta/DeltaBinaryEncoderLongTest.java  |    7 -
 .../iotdb/tsfile/read/ReadOnlyTsFileTest.java      |    4 +-
 .../iotdb/tsfile/read/TimePlainEncodeReadTest.java |    2 +-
 .../tsfile/read/TsFileSequenceReaderTest.java      |   10 -
 .../read/filter/IExpressionOptimizerTest.java      |   14 +-
 .../read/query/executor/QueryExecutorTest.java     |    2 +-
 .../query/timegenerator/ReaderByTimestampTest.java |    6 +-
 .../query/timegenerator/TimeGeneratorTest.java     |    2 +-
 .../iotdb/tsfile/read/reader/ReaderTest.java       |    2 +-
 .../org/apache/iotdb/tsfile/utils/RecordUtils.java |    2 -
 .../iotdb/tsfile/utils/TsFileGeneratorForTest.java |   23 +-
 .../iotdb/tsfile/write/TsFileIOWriterTest.java     |    5 +-
 .../iotdb/tsfile/write/TsFileReadWriteTest.java    |    4 +-
 ...Test.java => RestorableTsFileIOWriterTest.java} |   93 +-
 462 files changed, 12574 insertions(+), 22164 deletions(-)

diff --git a/.gitignore b/.gitignore
index a674f1f..d8c86d0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,8 +7,8 @@ iotdb/iotdb/data/*
 iotdb/iotdb/gc.log*
 iotdb/iotdb/logs/*
 iotdb/iotdb/lib/*
-iotdb/overflow/
-iotdb/testoverflowfile
+iotdb/unsequence/
+iotdb/testunsequence
 
 tsfile/src/test/resources/perTestInputData
 # Eclipse IDE files
@@ -80,6 +80,8 @@ grafana/logs/
 **/.nb-gradle/
 grafana/data/
 
+### vscode project
+**/.vscode/
 
 **/.DS_Store
 
diff --git a/.mvn/wrapper/MavenWrapperDownloader.java b/.mvn/wrapper/MavenWrapperDownloader.java
index a22fe2e..81453c4 100644
--- a/.mvn/wrapper/MavenWrapperDownloader.java
+++ b/.mvn/wrapper/MavenWrapperDownloader.java
@@ -45,9 +45,7 @@ public class MavenWrapperDownloader {
   private static final String PROPERTY_NAME_WRAPPER_URL = "wrapperUrl";
 
   public static void main(String args[]) {
-    System.out.println("- Downloader started");
     File baseDirectory = new File(args[0]);
-    System.out.println("- Using base directory: " + baseDirectory.getAbsolutePath());
 
     // If the maven-wrapper.properties exists, read it and check if it contains a custom
     // wrapperUrl parameter.
@@ -61,7 +59,6 @@ public class MavenWrapperDownloader {
         mavenWrapperProperties.load(mavenWrapperPropertyFileInputStream);
         url = mavenWrapperProperties.getProperty(PROPERTY_NAME_WRAPPER_URL, url);
       } catch (IOException e) {
-        System.out.println("- ERROR loading '" + MAVEN_WRAPPER_PROPERTIES_PATH + "'");
       } finally {
         try {
           if (mavenWrapperPropertyFileInputStream != null) {
@@ -72,7 +69,6 @@ public class MavenWrapperDownloader {
         }
       }
     }
-    System.out.println("- Downloading from: : " + url);
 
     File outputFile = new File(baseDirectory.getAbsolutePath(), MAVEN_WRAPPER_JAR_PATH);
     if (!outputFile.getParentFile().exists()) {
diff --git a/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md b/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md
index d38a2dd..ab04692 100644
--- a/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md	
+++ b/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md	
@@ -211,7 +211,7 @@ In total, we provide users three kinds of configurations module:
 * environment configuration file (iotdb-env.bat, iotdb-env.sh). The default configuration file for the environment configuration item. Users can configure the relevant system configuration items of JAVA-JVM in the file.
 * system configuration file (tsfile-format.properties, iotdb-engine.properties). 
 	* tsfile-format.properties: The default configuration file for the IoTDB file layer configuration item. Users can configure the information about the TsFile, such as the data size written to the disk per time(group\_size\_in_byte). 
-	* iotdb-engine.properties: The default configuration file for the IoTDB engine layer configuration item. Users can configure the IoTDB engine related parameters in the file, such as JDBC service listening port (rpc\_port), overflow data storage directory (overflow\_data\_dir), etc.
+	* iotdb-engine.properties: The default configuration file for the IoTDB engine layer configuration item. Users can configure the IoTDB engine related parameters in the file, such as JDBC service listening port (rpc\_port), unsequence data storage directory (unsequence\_data\_dir), etc.
 * log configuration file (logback.xml)
 
 The configuration files of the three configuration items are located in the IoTDB installation directory: $IOTDB_HOME/conf folder.
@@ -281,6 +281,15 @@ The detail of each variables are as follows:
 |Default| 134217728 |
 |Effective|Immediately|
 
+* page\_size\_in\_byte
+
+|Name| page\_size\_in\_byte |
+|:---:|:---|
+|Description|The maximum size of a single page written in memory when each column in memory is written (in bytes)|
+|Type|Int32|
+|Default| 134217728 |
+|Effective|Immediately|
+
 * max\_number\_of\_points\_in\_page
 
 |Name| max\_number\_of\_points\_in\_page |
@@ -299,15 +308,6 @@ The detail of each variables are as follows:
 |Default| 128 |
 |Effective|Immediately|
 
-* page\_size\_in\_byte
-
-|Name| page\_size\_in\_byte |
-|:---:|:---|
-|Description|The maximum size of a single page written in memory when each column in memory is written (in bytes)|
-|Type|Int32|
-|Default| 134217728 |
-|Effective|Immediately|
-
 * time\_series\_data\_type
 
 |Name| time\_series\_data\_type |
@@ -317,15 +317,24 @@ The detail of each variables are as follows:
 |Default| Int64 |
 |Effective|Immediately|
 
-* time\_series\_encoder
+* time\_encoder
 
-|Name| time\_series\_data\_type |
+|Name| time\_encoder |
 |:---:|:---|
-|Description| TimeSeries encoding type|
+|Description| Encoding type of time column|
 |Type|Enum String: “TS_2DIFF”,“PLAIN”,“RLE”|
 |Default| TS_2DIFF |
 |Effective|Immediately|
 
+* value\_encoder
+
+|Name| value\_encoder |
+|:---:|:---|
+|Description| Encoding type of value column|
+|Type|Enum String: “TS_2DIFF”,“PLAIN”,“RLE”|
+|Default| PLAIN |
+|Effective|Immediately|
+
 * float_precision
 
 |Name| float_precision |
@@ -337,76 +346,67 @@ The detail of each variables are as follows:
 
 #### Engine Layer
 
-* back\_loop\_period
-
-|Name| back\_loop\_period |
-|:---:|:---|
-|Description| The frequency at which the system statistic module triggers(in seconds). |
-|Type|Int32|
-|Default| 10 |
-|Effective|After restart system|
-
-* data\_dir
+* rpc_address
 
-|Name| data\_dir |
+|Name| rpc_address |
 |:---:|:---|
-|Description| The IoTDB data path.By default, it is stored in the data directory at the same level as the bin directory. It is recommended to use an absolute path. |
+|Description| The jdbc service listens on the address.|
 |Type|String|
-|Default| data |
+|Default| "0.0.0.0" |
 |Effective|After restart system|
 
-* enable_wal
+* rpc_port
 
-|Name| enable_wal |
+|Name| rpc_port |
 |:---:|:---|
-|Description| Whether to enable the pre-write log. The default value is true(enabled), and false means closed. |
-|Type|Bool|
-|Default| true |
+|Description| The jdbc service listens on the port. Please confirm that the port is not a system reserved port and is not occupied.|
+|Type|Short Int : [0,65535]|
+|Default| 6667 |
 |Effective|After restart system|
 
-* fetch_size
+* time_zone
 
-|Name| fetch_size |
+|Name| time_zone |
 |:---:|:---|
-|Description| The amount of data read each time in batches(the number of data strips, that is, the number of different time stamps.) |
-|Type|Int32|
-|Default| 10000 |
+|Description| The time zone in which the server is located, the default is Beijing time (+8) |
+|Type|Time Zone String|
+|Default| +08:00 |
 |Effective|After restart system|
 
-* flush\_wal\_period\_in\_ms
+* base\_dir
 
-|Name| flush\_wal\_period\_in\_ms |
+|Name| base\_dir |
 |:---:|:---|
-|Description| The period during which the log is periodically flushed to disk(in milliseconds) |
-|Type|Int32|
-|Default| 10 |
+|Description| The IoTDB system folder. It is recommended to use an absolute path. |
+|Type|String|
+|Default| data |
 |Effective|After restart system|
 
-* flush\_wal\_threshold
+* data_dirs
 
-|Name| flush\_wal\_threshold |
+|Name| data_dirs |
 |:---:|:---|
-|Description| After the WAL reaches this value, it is flushed to disk, and it is possible to lose at most flush_wal_threshold operations. |
-|Type|Int32|
-|Default| 10000 |
+|Description| The directories of data files. Multiple directories are separated by comma. See the [mult\_dir\_strategy](chapter4,multdirstrategy) configuration item for data distribution strategy. The starting directory of the relative path is related to the operating system. It is recommended to use an absolute path. If the path does not exist, the system will automatically create it.|
+|Type|String[]|
+|Default| data/data |
 |Effective|After restart system|
 
-* max\_opened\_folder
+* wal\_dir
 
-|Name| max\_opened\_folder |
+|Name| wal\_dir |
 |:---:|:---|
-|Description| The maximum number of folders opened at the same time. When the value becomes larger, the memory usage increases, the IO random read and write becomes less, and the file partition (ie, group) is more neat; the smaller the value, the less memory is occupied, the IO random read and write becomes more, and the file block size is insufficient. |
-|Type|Int32|
-|Default| 100 |
+|Description| Write Ahead Log storage path. It is recommended to use an absolute path. |
+|Type|String|
+|Default| data/wal |
 |Effective|After restart system|
 
-* merge\_concurrent\_threads
+* enable_wal
 
-|Name| merge\_concurrent\_threads |
+|Name| enable_wal |
 |:---:|:---|
-|Description| THe max threads which can be used when overflow data is merged. The larger it is, the more IO and CPU cost. The smaller the value, the more the disk is occupied when the overflow data is too large, the reading will be slower. |
-|Type|Int32|
-|Default| 10 |
+|Description| Whether to enable the pre-write log. The default value is true(enabled), and false means closed. |
+|Type|Bool|
+|Default| true |
 |Effective|After restart system|
 
 * mult\_dir\_strategy
@@ -418,76 +418,49 @@ The detail of each variables are as follows:
 |Default| MaxDiskUsableSpaceFirstStrategy |
 |Effective|After restart system|
 
-* period\_time\_for\_flush\_in\_second
+* tsfile\_size\_threshold
 
-|Name| period\_time\_for\_flush\_in\_second |
+|Name| tsfile\_size\_threshold |
 |:---:|:---|
-|Description| The interval period IoTDB closes files(in seconds). At every set time, the system will automatically flush the data in the memory to the disk and seal all the files that are currently open.|
-|Type|Int32|
-|Default| 3600 |
+|Description| When a TsFile size on the disk exceeds this threshold, the TsFile is closed and open a new TsFile to accept data writes. The unit is byte and the default value is 2G.|
+|Type| Int64 |
+|Default| 536870912 |
 |Effective|After restart system|
 
-* period\_time\_for\_merge\_in\_second
+* flush\_wal\_threshold
 
-|Name| period\_time\_for\_merge\_in\_second |
+|Name| flush\_wal\_threshold |
 |:---:|:---|
-|Description| IoTDB has two parts of data in memory at runtime: overflow and bufferwrite. The system will automatically merge the two parts of data at regular intervals. This is the merge interval(in seconds).|
+|Description| After the WAL reaches this value, it is flushed to disk, and it is possible to lose at most flush_wal_threshold operations. |
 |Type|Int32|
-|Default| 7200 |
-|Effective|After restart system|
-
-* rpc_address
-
-|Name| rpc_address |
-|:---:|:---|
-|Description| The jdbc service listens on the address.|
-|Type|String|
-|Default| "0.0.0.0" |
-|Effective|After restart system|
-
-* rpc_port
-
-|Name| rpc_port |
-|:---:|:---|
-|Description| The jdbc service listens on the port. Please confirm that the port is not a system reserved port and is not occupied.|
-|Type|Short Int : [0,65535]|
-|Default| 6667 |
-|Effective|After restart system|
-
-* tsfile_dir
-
-|Name| tsfile_dir |
-|:---:|:---|
-|Description| The storage path of TsFile. By default, it is stored in three folders under data directory(soldled1, settled2, and settled3). See the [mult\_dir\_strategy](chapter4,multdirstrategy) configuration item for data distribution strategy. The starting directory of the relative path is related to the operating system. It is recommended to use an absolute path. If the path does not exist, the system will automatically create it.|
-|Type|String[]|
-|Default| settled1, settled2, settled3 |
+|Default| 10000 |
 |Effective|After restart system|
 
-* wal\_cleanup\_threshold
+* flush\_wal\_period\_in\_ms
 
-|Name| wal\_cleanup\_threshold |
+|Name| force\_wal\_period\_in\_ms |
 |:---:|:---|
-|Description| When the total number of logs in the file and in memory reaches this value, all logs are compressed and the useless log is removed. The default is 500000. If this value is too large, it will cause a short write pause. If it is too small, it will increase IO and CPU consumption. |
+|Description| The period during which the log is periodically forced to flush to disk(in milliseconds) |
 |Type|Int32|
-|Default| 500000 |
+|Default| 10 |
 |Effective|After restart system|
 
-* sys\_dir
+* fetch_size
 
-|Name| sys\_dir |
+|Name| fetch_size |
 |:---:|:---|
-|Description| IoTDB metadata storage path.(By default it is in the data directory at the same level as the bin directory. The starting directory of the relative path is related to the operating system. It is recommended to use an absolute path. |
-|Type|String|
-|Default| system |
+|Description| The amount of data read each time in batch (the number of data strips, that is, the number of different timestamps.) |
+|Type|Int32|
+|Default| 10000 |
 |Effective|After restart system|
 
-* time_zone
+* merge\_concurrent\_threads
 
-|Name| time_zone |
+|Name| merge\_concurrent\_threads |
 |:---:|:---|
-|Description| The time zone in which the server is located, the default is Beijing time (+8) |
-|Type|Time Zone String|
-|Default| +08:00 |
+|Description| THe max threads which can be used when unsequence data is merged. The larger it is, the more IO and CPU cost. The smaller the value, the more the disk is occupied when the unsequence data is too large, the reading will be slower. |
+|Type|Int32|
+|Default| 10 |
 |Effective|After restart system|
 
 * enable\_stat\_monitor
@@ -499,70 +472,13 @@ The detail of each variables are as follows:
 |Default| true |
 |Effective|After restart system|
 
+* back\_loop\_period_in_second
 
-* mem\_threshold\_warning
-
-|Name| mem\_threshold\_warning |
-|:---:|:---|
-|Description| A percentage value, which is multiplied by the maximum heap memory assigned by the IoTDB runtime to get a threshold. When the IoTDB uses memory beyond the threshold, it will trigger the operation of writing the current in-memory data to the disk and releasing the corresponding memory. By default, IoTDB runtime can use 80% of the maximum heap memory. If the value is configured to exceed 1, the configuration item will not take effect. If the value is less than or equal to 0,  [...]
-|Type| Float |
-|Default| 0.8 |
-|Effective|After restart system|
-
-
-* mem\_threshold\_dangerous
-
-|Name| mem\_threshold\_dangerous |
+|Name| back\_loop\_period\_in\_second |
 |:---:|:---|
-|Description| A percentage value, which is multiplied by the maximum heap memory allocated by the IoTDB runtime to get a threshold. When the IoTDB uses memory beyond the threshold, it will trigger the operation of writing the current in-memory data to the disk and releasing the corresponding memory. At the same time, the write operation will be blocked. By default, the IoTDB runtime can use 90% of the maximum heap memory. If the value is configured to exceed 1, the configuration item wil [...]
-|Type| Float |
-|Default| 0.9 |
-|Effective|After restart system|
-
-* mem\_monitor\_interval
-
-|Name| mem\_monitor\_interval |
-|:---:|:---|
-|Description| The time interval IoTDB system checks the current memory usage. If the threshold calculated according to mem_threshold_warning or mem_threshold_dangerous is exceeded, the corresponding operation will be triggered. The unit is milliseconds and the default is 1000 milliseconds.|
-|Type| Int64 |
-|Default| 1000 |
-|Effective|After restart system|
-
-* bufferwrite\_meta\_size\_threshold
-
-|Name| bufferwrite\_meta\_size\_threshold |
-|:---:|:---|
-|Description| When the metadata size of the TsFile saved in the memory exceeds the threshold, the metadata is saved at the end of the TsFile, and then the file is closed and the memory space occupied by the metadata is released. The unit is byte and the default value is 200M.|
-|Type| Int64 |
-|Default| 209715200 |
-|Effective|After restart system|
-
-* bufferwrite\_file\_size\_threshold
-
-|Name| bufferwrite\_meta\_size\_threshold |
-|:---:|:---|
-|Description| When a TsFile size on the disk exceeds this threshold, the TsFile is closed and open a new TsFile to accept data writes. The unit is byte and the default value is 2G.|
-|Type| Int64 |
-|Default| 2147483648 |
-|Effective|After restart system|
-
-
-* overflow\_meta\_size\_threshold
-
-|Name| overflow\_meta\_size\_threshold |
-|:---:|:---|
-|Description| When the size of the Overflow metadata stored in the memory exceeds the threshold, the metadata is saved at the end of the TsFile, then the file is closed and the memory space occupied by the metadata is released. The unit is byte and the default value is 200M.|
-|Type| Int64 |
-|Default| 209715200 |
-|Effective|After restart system|
-
-* overflow\_file\_size\_threshold
-
-|Name| overflow\_file\_size\_threshold |
-|:---:|:---|
-|Description| When an Overflow file size on the disk exceeds this threshold, the Overflow file is closed. And open a new Overflow file to accept data writes. The unit is byte, the default value is 2G|
-|Type| Int64 |
-|Default| 2147483648 |
+|Description| The frequency at which the system statistic module triggers(in seconds). |
+|Type|Int32|
+|Default| 5 |
 |Effective|After restart system|
 
 * concurrent\_flush\_thread
@@ -574,7 +490,6 @@ The detail of each variables are as follows:
 |Default| 0 |
 |Effective|After restart system|
 
-
 * stat\_monitor\_detect\_freq\_sec
 
 |Name| concurrent\_flush\_thread |
@@ -584,7 +499,6 @@ The detail of each variables are as follows:
 |Default|600 |
 |Effective|After restart system|
 
-
 * stat\_monitor\_retain\_interval\_sec
 
 |Name| stat\_monitor\_retain\_interval\_sec |
@@ -672,14 +586,6 @@ There are several attributes under Monitor, including the numbers of files opene
 |Default Directory| /data/data/settled |
 |Type| Int |
 
-* OverflowOpenFileNum
-
-|Name| OverflowOpenFileNum |
-|:---:|:---|
-|Description| The opened out-of-order data file number of IoTDB server process. |
-|Default Directory| /data/data/overflow |
-|Type| Int |
-
 * WalOpenFileNum
 
 |Name| WalOpenFileNum |
@@ -901,7 +807,7 @@ Here are the file size statistics:
 
 |Name| OVERFLOW |
 |:---:|:---|
-|Description| Calculate the sum of all the ```out-of-order data file``` size (under ```data/data/overflow``` by default) in byte.|
+|Description| Calculate the sum of all the ```out-of-order data file``` size (under ```data/data/unsequence``` by default) in byte.|
 |Type| File size statistics |
 |Timeseries Name| root.stats.file\_size.OVERFLOW |
 |Reset After Restarting System| No |
diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java
index 3e5d180..f1fdc9f 100644
--- a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java
+++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java
@@ -40,7 +40,7 @@ public class KafkaConsumerThread implements Runnable {
   private String createStorageGroupSqlTemplate = "SET STORAGE GROUP TO %s";
   private String createTimeseriesSqlTemplate = "CREATE TIMESERIES %s WITH DATATYPE=TEXT, ENCODING=PLAIN";
   private String insertDataSqlTemplate = "INSERT INTO root.vehicle.device(timestamp,%s) VALUES (%s,'%s')";
-  private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerThread.class);
+  private static final Logger logger = LoggerFactory.getLogger(KafkaConsumerThread.class);
 
   public KafkaConsumerThread(KafkaStream<String, String> stream) {
     this.stream = stream;
@@ -73,7 +73,7 @@ public class KafkaConsumerThread implements Runnable {
         createTimeSeries = false;
       }
     } catch (ClassNotFoundException | SQLException e) {
-      LOGGER.error(e.getMessage());
+      logger.error(e.getMessage());
     }
   }
 
@@ -88,14 +88,14 @@ public class KafkaConsumerThread implements Runnable {
       String sql = String.format(insertDataSqlTemplate, items[0], items[1], items[2]);
       statement.execute(sql);
     } catch (SQLException e) {
-      LOGGER.error(e.getMessage());
+      logger.error(e.getMessage());
     }
   }
 
   public void run() {
     for (MessageAndMetadata<String, String> consumerIterator : stream) {
       String uploadMessage = consumerIterator.message();
-      LOGGER.info(String.format("%s from partiton[%d]: %s", Thread.currentThread().getName(),
+      logger.info(String.format("%s from partiton[%d]: %s", Thread.currentThread().getName(),
           consumerIterator.partition(), uploadMessage));
       writeData(uploadMessage);
     }
diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java
index ec8d8b4..ac56a26 100644
--- a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java
+++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory;
 public class KafkaProducer {
 
   private final Producer<String, String> producer;
-  private static final Logger LOGGER = LoggerFactory.getLogger(KafkaProducer.class);
+  private static final Logger logger = LoggerFactory.getLogger(KafkaProducer.class);
 
   public KafkaProducer() {
 
@@ -57,7 +57,7 @@ public class KafkaProducer {
     for (int i = 0; i < Constant.ALL_DATA.length; i++) {
       String key = Integer.toString(i);
       producer.send(new KeyedMessage<>(Constant.TOPIC, key, Constant.ALL_DATA[i]));
-      LOGGER.info(Constant.ALL_DATA[i]);
+      logger.info(Constant.ALL_DATA[i]);
     }
   }
 
diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java
index 255a418..2fbda07 100644
--- a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java
+++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java
@@ -41,7 +41,7 @@ public class RocketMQConsumer {
   private Connection connection;
   private Statement statement;
   private String createStorageGroupSqlTemplate = "SET STORAGE GROUP TO %s";
-  private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQConsumer.class);
+  private static final Logger logger = LoggerFactory.getLogger(RocketMQConsumer.class);
 
   public RocketMQConsumer(String producerGroup, String serverAddresses, String connectionUrl,
       String user, String password) throws ClassNotFoundException, SQLException {
@@ -94,12 +94,12 @@ public class RocketMQConsumer {
      * Register callback to execute on arrival of messages fetched from brokers.
      */
     consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> {
-      LOGGER.info(String.format("%s Receive New Messages: %s %n", Thread.currentThread().getName(),
+      logger.info(String.format("%s Receive New Messages: %s %n", Thread.currentThread().getName(),
           new String(msgs.get(0).getBody())));
       try {
         statement.execute(new String(msgs.get(0).getBody()));
       } catch (SQLException e) {
-        LOGGER.error(e.getMessage());
+        logger.error(e.getMessage());
       }
       return ConsumeOrderlyStatus.SUCCESS;
     });
diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java
index f4c5a07..2e231d9 100644
--- a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java
+++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java
@@ -38,7 +38,7 @@ public class RocketMQProducer {
   private DefaultMQProducer producer;
   private String producerGroup;
   private String serverAddresses;
-  private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQProducer.class);
+  private static final Logger logger = LoggerFactory.getLogger(RocketMQProducer.class);
 
   public RocketMQProducer(String producerGroup, String serverAddresses) {
     this.producerGroup = producerGroup;
@@ -65,7 +65,7 @@ public class RocketMQProducer {
         return mqs.get(index);
       }, Utils
           .ConvertStringToInteger(Utils.getTimeSeries(sql)));
-      LOGGER.info(sendResult.toString());
+      logger.info(sendResult.toString());
     }
   }
 
diff --git a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFInputFormat.java b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFInputFormat.java
index 45298b3..29c8071 100644
--- a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFInputFormat.java
+++ b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFInputFormat.java
@@ -78,7 +78,7 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
    * key to configure the reading measurementIds
    */
   public static final String READ_MEASUREMENTID = "tsfile.read.measurement";
-  private static final Logger LOGGER = LoggerFactory.getLogger(TSFInputFormat.class);
+  private static final Logger logger = LoggerFactory.getLogger(TSFInputFormat.class);
   private static final String SPERATOR = ",";
 
   /**
@@ -252,10 +252,10 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
     List<InputSplit> splits = new ArrayList<>();
     // get the all file in the directory
     List<FileStatus> listFileStatus = super.listStatus(job);
-    LOGGER.info("The number of this job file is {}", listFileStatus.size());
+    logger.info("The number of this job file is {}", listFileStatus.size());
     // For each file
     for (FileStatus fileStatus : listFileStatus) {
-      LOGGER.info("The file path is {}", fileStatus.getPath());
+      logger.info("The file path is {}", fileStatus.getPath());
       // Get the file path
       Path path = fileStatus.getPath();
       // Get the file length
@@ -265,27 +265,26 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
       if (length > 0) {
         // Get block information in the local file system or hdfs
         if (fileStatus instanceof LocatedFileStatus) {
-          LOGGER.info("The file status is {}", LocatedFileStatus.class.getName());
+          logger.info("The file status is {}", LocatedFileStatus.class.getName());
           blockLocations = ((LocatedFileStatus) fileStatus).getBlockLocations();
         } else {
           FileSystem fileSystem = path.getFileSystem(configuration);
-          LOGGER.info("The file status is {}", fileStatus.getClass().getName());
-          System.out.println("The file status is " + fileStatus.getClass().getName());
-          System.out.println("The file system is " + fileSystem.getClass());
+          logger.info("The file status is {}", fileStatus.getClass().getName());
+          logger.info("The file system is " + fileSystem.getClass());
           blockLocations = fileSystem.getFileBlockLocations(fileStatus, 0, length);
         }
-        LOGGER.info("The block location information is {}", Arrays.toString(blockLocations));
+        logger.info("The block location information is {}", Arrays.toString(blockLocations));
         HDFSInputStream hdfsInputStream = new HDFSInputStream(path, configuration);
         FileReader fileReader = new FileReader(hdfsInputStream);
         // Get the timeserise to test
         splits.addAll(generateSplits(path, fileReader, blockLocations));
         fileReader.close();
       } else {
-        LOGGER.warn("The file length is " + length);
+        logger.warn("The file length is " + length);
       }
     }
     configuration.setLong(NUM_INPUT_FILES, listFileStatus.size());
-    LOGGER.info("The number of splits is " + splits.size());
+    logger.info("The number of splits is " + splits.size());
 
     return splits;
   }
@@ -320,7 +319,7 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
     long splitStart = 0;
     List<String> hosts = new ArrayList<>();
     for (RowGroupMetaData rowGroupMetaData : fileReader.getSortedRowGroupMetaDataList()) {
-      LOGGER.info("The rowGroupMetaData information is {}", rowGroupMetaData);
+      logger.info("The rowGroupMetaData information is {}", rowGroupMetaData);
 
       long start = getRowGroupStart(rowGroupMetaData);
       int blkIndex = getBlockLocationIndex(blockLocations, start);
@@ -332,7 +331,7 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
       if (blkIndex != currentBlockIndex) {
         TSFInputSplit tsfInputSplit = makeSplit(path, rowGroupMetaDataList, splitStart,
             splitSize, hosts);
-        LOGGER.info("The tsfile inputsplit information is {}", tsfInputSplit);
+        logger.info("The tsfile inputsplit information is {}", tsfInputSplit);
         splits.add(tsfInputSplit);
 
         currentBlockIndex = blkIndex;
@@ -348,7 +347,7 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
     }
     TSFInputSplit tsfInputSplit = makeSplit(path, rowGroupMetaDataList, splitStart,
         splitSize, hosts);
-    LOGGER.info("The tsfile inputsplit information is {}", tsfInputSplit);
+    logger.info("The tsfile inputsplit information is {}", tsfInputSplit);
     splits.add(tsfInputSplit);
     return splits;
   }
@@ -364,7 +363,7 @@ public class TSFInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
         return i;
       }
     }
-    LOGGER.warn(String.format("Can't find the block. The start is:%d. the last block is", start),
+    logger.warn(String.format("Can't find the block. The start is:%d. the last block is", start),
         blockLocations[blockLocations.length - 1].getOffset()
             + blockLocations[blockLocations.length - 1].getLength());
     return -1;
diff --git a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFOutputFormat.java b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFOutputFormat.java
index 9a501d0..fbf290b 100644
--- a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFOutputFormat.java
@@ -34,19 +34,19 @@ import org.slf4j.LoggerFactory;
 public class TSFOutputFormat extends FileOutputFormat<NullWritable, TSRow> {
 
   public static final String FILE_SCHEMA = "tsfile.schema";
-  private static final Logger LOGGER = LoggerFactory.getLogger(TSFOutputFormat.class);
+  private static final Logger logger = LoggerFactory.getLogger(TSFOutputFormat.class);
   private static final String extension = "tsfile";
 
   public static void setWriterSchema(Job job, JSONObject schema) {
 
-    LOGGER.info("Set the write schema - {}", schema.toString());
+    logger.info("Set the write schema - {}", schema.toString());
 
     job.getConfiguration().set(FILE_SCHEMA, schema.toString());
   }
 
   public static void setWriterSchema(Job job, String schema) {
 
-    LOGGER.info("Set the write schema - {}", schema);
+    logger.info("Set the write schema - {}", schema);
 
     job.getConfiguration().set(FILE_SCHEMA, schema);
   }
@@ -67,7 +67,7 @@ public class TSFOutputFormat extends FileOutputFormat<NullWritable, TSRow> {
       throws IOException, InterruptedException {
 
     Path outputPath = getDefaultWorkFile(job, extension);
-    LOGGER.info("The task attempt id is {}, the output path is {}", job.getTaskAttemptID(),
+    logger.info("The task attempt id is {}, the output path is {}", job.getTaskAttemptID(),
         outputPath);
     JSONObject schema = getWriterSchema(job);
     return new TSFRecordWriter(outputPath, schema);
diff --git a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordReader.java b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordReader.java
index 560d764..4e125b4 100644
--- a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordReader.java
+++ b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordReader.java
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  */
 public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TSFRecordReader.class);
+  private static final Logger logger = LoggerFactory.getLogger(TSFRecordReader.class);
 
   private QueryDataSet dataSet = null;
   private List<Field> fields = null;
@@ -84,8 +84,8 @@ public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
       if (measurementIdsList == null) {
         measurementIdsList = initSensorIdList(rowGroupMetaDataList);
       }
-      LOGGER.info("deltaObjectIds:" + deltaObjectIdsList);
-      LOGGER.info("Sensors:" + measurementIdsList);
+      logger.info("deltaObjectIds:" + deltaObjectIdsList);
+      logger.info("Sensors:" + measurementIdsList);
 
       this.sensorNum = measurementIdsList.size();
       isReadDeviceId = TSFInputFormat.getReadDeltaObject(configuration);
@@ -102,7 +102,7 @@ public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
       dataSet = queryEngine
           .queryWithSpecificRowGroups(deltaObjectIdsList, measurementIdsList, null, null, null);
     } else {
-      LOGGER.error("The InputSplit class is not {}, the class is {}", TSFInputSplit.class.getName(),
+      logger.error("The InputSplit class is not {}, the class is {}", TSFInputSplit.class.getName(),
           split.getClass().getName());
       throw new InternalError(String.format("The InputSplit class is not %s, the class is %s",
           TSFInputSplit.class.getName(), split.getClass().getName()));
@@ -133,9 +133,9 @@ public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
     sensorIndex += sensorNum;
 
     if (fields == null || sensorIndex >= fields.size()) {
-      LOGGER.info("Start another row~");
+      logger.info("Start another row~");
       if (!dataSet.next()) {
-        LOGGER.info("Finish all rows~");
+        logger.info("Finish all rows~");
         return false;
       }
 
@@ -176,7 +176,7 @@ public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
     for (int i = 0; i < sensorNum; i++) {
       Field field = fields.get(sensorIndex + i);
       if (field.isNull()) {
-        LOGGER.info("Current value is null");
+        logger.info("Current value is null");
         writables[index] = NullWritable.get();
       } else {
         switch (field.dataType) {
@@ -199,7 +199,7 @@ public class TSFRecordReader extends RecordReader<NullWritable, ArrayWritable> {
             writables[index] = new Text(field.getBinaryV().getStringValue());
             break;
           default:
-            LOGGER.error("The data type is not support {}", field.dataType);
+            logger.error("The data type is not support {}", field.dataType);
             throw new InterruptedException(
                 String.format("The data type %s is not support ", field.dataType));
         }
diff --git a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordWriter.java b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordWriter.java
index ad88824..3f23cf6 100644
--- a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordWriter.java
+++ b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/TSFRecordWriter.java
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 
 public class TSFRecordWriter extends RecordWriter<NullWritable, TSRow> {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TSFRecordWriter.class);
+  private static final Logger logger = LoggerFactory.getLogger(TSFRecordWriter.class);
 
   private TsFile write = null;
 
@@ -70,7 +70,7 @@ public class TSFRecordWriter extends RecordWriter<NullWritable, TSRow> {
   @Override
   public void close(TaskAttemptContext context) throws IOException, InterruptedException {
 
-    LOGGER.info("Close the recordwriter, the task attempt id is {}", context.getTaskAttemptID());
+    logger.info("Close the recordwriter, the task attempt id is {}", context.getTaskAttemptID());
     write.close();
   }
 
diff --git a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/example/TsFileHelper.java b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/example/TsFileHelper.java
index 23dcd48..0ca01a9 100644
--- a/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/example/TsFileHelper.java
+++ b/hadoop/src/main/java/org/apache/iotdb/tsfile/hadoop/example/TsFileHelper.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
 
 public class TsFileHelper {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TsFileHelper.class);
+  private static final Logger logger = LoggerFactory.getLogger(TsFileHelper.class);
 
   public static void deleteTsFile(String filePath) {
     File file = new File(filePath);
@@ -121,8 +121,8 @@ public class TsFileHelper {
     file.delete();
     writeTsFile(filePath);
     TsFile tsFile = new TsFile(new TsRandomAccessLocalFileReader(filePath));
-    LOGGER.info("Get columns information: {}", tsFile.getAllColumns());
-    LOGGER.info("Get all deltaObjectId: {}", tsFile.getAllDeltaObject());
+    logger.info("Get columns information: {}", tsFile.getAllColumns());
+    logger.info("Get all deltaObjectId: {}", tsFile.getAllDeltaObject());
     tsFile.close();
   }
 }
diff --git a/iotdb-cli/src/test/java/org/apache/iotdb/cli/tool/ImportCsvTestIT.java b/iotdb-cli/src/test/java/org/apache/iotdb/cli/tool/ImportCsvTestIT.java
index 13b573f..57812c4 100644
--- a/iotdb-cli/src/test/java/org/apache/iotdb/cli/tool/ImportCsvTestIT.java
+++ b/iotdb-cli/src/test/java/org/apache/iotdb/cli/tool/ImportCsvTestIT.java
@@ -67,7 +67,6 @@ public class ImportCsvTestIT extends AbstractScript {
         "Encounter an error when importing data, error is: Connection Error, please check whether "
             + "the network is available or the server has started."};
     String dir = getCurrentPath("pwd");
-    System.out.println(dir);
     ProcessBuilder builder = new ProcessBuilder("sh",
         dir + File.separator + "cli" + File.separator + "bin" + File.separator + "import-csv.sh",
         "-h",
diff --git a/iotdb/iotdb/conf/iotdb-engine.properties b/iotdb/iotdb/conf/iotdb-engine.properties
index 848696c..0466025 100644
--- a/iotdb/iotdb/conf/iotdb-engine.properties
+++ b/iotdb/iotdb/conf/iotdb-engine.properties
@@ -29,19 +29,15 @@ rpc_port=6667
 ### Write Ahead Log Configuration
 ####################
 
-# Is write ahead log enable
+# Is insert ahead log enable
 enable_wal=true
 
-# When a certain amount of write ahead log is reached, it will be flushed to disk
+# When a certain amount of insert ahead log is reached, it will be flushed to disk
 # It is possible to lose at most flush_wal_threshold operations
 flush_wal_threshold=10000
 
-# The cycle when write ahead log is periodically refreshed to disk(in milliseconds)
-# It is possible to lose at most flush_wal_period_in_ms ms operations
-flush_wal_period_in_ms=10
-
-# The cycle when write ahead log is periodically forced to be written to disk(in milliseconds)
-# If force_wal_period_in_ms = 0 it means force write ahead log to be written to disk after each refreshment
+# The cycle when insert ahead log is periodically forced to be written to disk(in milliseconds)
+# If force_wal_period_in_ms = 0 it means force insert ahead log to be written to disk after each refreshment
 # Set this parameter to 0 may slow down the ingestion on slow disk.
 force_wal_period_in_ms=10
 
@@ -49,30 +45,31 @@ force_wal_period_in_ms=10
 ### Directory Configuration
 ####################
 
-# data dir
-# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data/data).
+# base dir
+# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data).
 # If it is absolute, system will save the data in exact location it points to.
 # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
-# Note: If data_dir is assigned an empty string(i.e.,zero-length), it will be handled as a relative path.
+# Note: If sys_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
 # For windows platform
 # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
-# data_dir=D:\\iotdb\\data\\data
+# base_dir=data
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# data_dir=/path/iotdb/data/data
-
-# tsfile dir
-# For this property, multiple directories should be set, and all directories should be separated by ",". All TsFiles will be allocated separately in all these directories. Moreover, setting absolute directories is suggested.
-# If this property is unset, system will save the TsFiles in the default relative path directory under the data_dir folder(i.e., %IOTDB_HOME%/data/data/settled).
-# If some are absolute, system will save the data in exact location they point to.
-# If some are relative, system will save the data in the relative path directory they indicate under the data_dir folder.
-# Note: If some are assigned an empty string(i.e.,zero-length), they will be handled as a relative path.
+# base_dir=data
+
+
+# data dirs
+# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data/data).
+# If it is absolute, system will save the data in exact location it points to.
+# If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
+# Note: If data_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
 # For windows platform
 # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
-# tsfile_dir=D:\\iotdb\\data\\data,data\\data
+# data_dirs=data\\data
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# tsfile_dir=/path/iotdb/data/data,data/data
+# data_dirs=data/data
+
 
 # mult_dir_strategy
 # The strategy is used to choose a directory from tsfile_dir for the system to store a new tsfile.
@@ -84,99 +81,56 @@ force_wal_period_in_ms=10
 # Set SequenceStrategy,MaxDiskUsableSpaceFirstStrategy and MinFolderOccupiedSpaceFirstStrategy to apply the corresponding strategy.
 # If this property is unset, system will use MaxDiskUsableSpaceFirstStrategy as default strategy.
 # For this property, fully-qualified class name (include package name) and simple class name are both acceptable.
-# mult_dir_strategy=MaxDiskUsableSpaceFirstStrategy
+# multi_dir_strategy=MaxDiskUsableSpaceFirstStrategy
 
-# system dir
-# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data/system).
-# If it is absolute, system will save the data in exact location it points to.
-# If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
-# Note: If sys_dir is assigned an empty string(i.e.,zero-length), it will be handled as a relative path.
-# For windows platform
-# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
-# sys_dir=D:\\iotdb\\data\\system
-# For Linux platform
-# If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# sys_dir=/path/iotdb/data/system
 
 # wal dir
 # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data).
 # If it is absolute, system will save the data in the exact location it points to.
 # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
-# Note: If wal_dir is assigned an empty string(i.e.,zero-length), it will be handled as a relative path.
+# Note: If wal_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
 # For windows platform
 # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
-# wal_dir=D:\\iotdb\\data
+# wal_dir=data\\wal
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# wal_dir=/path/iotdb/data
+# wal_dir=data/wal
 
 ####################
 ### Memory Control Configuration
 ####################
 
-# The maximum concurrent thread number for merging overflow
+# The maximum concurrent thread number for merging
 # Increase this value, it will increase IO and CPU consumption
-# Decrease this value, when there is much overflow data, it will increase disk usage, which will reduce read speed
+# Decrease this value, when there is much unsequence data, it will increase disk usage, which will reduce read speed
 # When the value<=0 or > CPU core number, use the CPU core number.
 merge_concurrent_threads=0
 
-# Maximum number of folders open at the same time
-# Increase this value, it will use more memory, random I/O becomes smaller, file fragmentation (i.e., group) is more neat.
-# Decrease this value, it will use less memory, random I/O becomes greater, file fragmentation is less neat.
-# group_size_in_byte * max_opened_folder = theoretical value of maximum memory occupancy
-# For an application, the total amount of folder is equal to the number of storage_group settings in SQL
-max_opened_folder=100
-
-# The amount of data that is read every time when IoTDB merge data.
+# The amount of data read each time in batch (the number of data strips, that is, the number of different timestamps.)
 fetch_size=10000
 
-# The period time of flushing data from memory to file.
-# The unit is second.
-period_time_for_flush_in_second=3600
-
-# The period time for merge overflow data with tsfile data.
-# The unit is second.
-period_time_for_merge_in_second=7200
+# Size of log buffer in each log node(in byte).
+# If WAL is enabled and the size of a insert plan is smaller than this parameter, then the insert plan will be rejected by WAL
+wal_buffer_size=16777216
 
-# When set to true, start timing flush and merge service. False, stop timing flush and merge service.
-# Default is true.
-enable_timing_close_and_merge=true
+# total number of memtables in memtable pool, should be set at least as twice of the number of storage groups.
+memtable_number=20
 
 # time zone of server side
 # default value is +08:00
 # eg. +08:00, -01:00
 time_zone=+08:00
 
-# if memory used by write reaches this threshold, auto flush will be triggered, percentile of Java heap memory
-mem_threshold_warning=0.5
-
-# if memory used by write reaches this threshold, write will be blocked, percentile of Java heap memory
-mem_threshold_dangerous=0.6
-
-# every such interval, a thread will check if memory exceeds mem_threshold_warning
-# if do exceed, auto flush will be triggered, the time unit is ms, 1s by default
-mem_monitor_interval_in_ms=1000
-
-# Decide how to control memory used by inserting data.
-# 0 is RecordMemController, which count the size of every record (tuple).
-# 1 is JVMMemController, which use JVM heap memory as threshold.
-# 2 is DisabledMemController, which does not control memory usage.
-mem_controller_type=0
-
-# When a bufferwrite's metadata size (in byte) exceed this, the bufferwrite is forced closed.
-bufferwrite_meta_size_threshold=209715200
-
-# When a bufferwrite's file size (in byte) exceeds this, the bufferwrite is forced closed.
-bufferwrite_file_size_threshold=2147483648
+# When a TsFile's file size (in byte) exceeds this, the TsFile is forced closed. The default threshold is 512 MB.
+tsfile_size_threshold=536870912
 
-# When a overflow's metadata size (in byte) exceeds this, the overflow is forced closed.
-overflow_meta_size_threshold=20971520
+# How many threads can concurrently flush. When <= 0, use CPU core number.
+concurrent_flush_thread=0
 
-# When a overflow's file size (in byte) exceeds this, the overflow is forced closed.
-overflow_file_size_threshold=209715200
+# whether take over the memory management by IoTDB rather than JVM when serializing memtable as bytes in memory
+# (i.e., whether use ChunkBufferPool), value true, false
+chunk_buffer_pool_enable = false
 
-# How many thread can concurrently flush. When <= 0, use CPU core number.
-concurrent_flush_thread=0
 
 ####################
 ### Statistics Monitor configuration
@@ -201,26 +155,10 @@ stat_monitor_detect_freq_in_second=600
 # The time unit is seconds.
 stat_monitor_retain_interval_in_second=600
 
-# When set false, MemMonitorThread and MemStatisticThread will not be created.
-enable_mem_monitor=true
-
-# When set to true, small flush will be triggered periodically even if memory threshold is not exceeded.
-enable_small_flush=true
-
-# The interval of small flush in ms.
-small_flush_interval=60000
-
-# The threshold of lines of external sort
-external_sort_threshold=50
-
 # cache size for MManager.
-# This cache is used to improve write speed where all path check and TSDataType will be cached in MManager with corresponding Path.
+# This cache is used to improve insert speed where all path check and TSDataType will be cached in MManager with corresponding Path.
 schema_manager_cache_size=300000
 
-# If the size in byte of a WAL log exceeds this, it won't be written.
-# Generally the default value 4MB is enough.
-max_log_entry_size=4194304
-
 ####################
 ### Sync Server Configuration
 ####################
@@ -235,11 +173,11 @@ sync_server_port=5555
 # Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
 # If there are more than one IP segment, please separate them by commas
 # The default is to allow all IP to sync
-IP_white_list=0.0.0.0/0
+ip_white_list=0.0.0.0/0
 
 # The processing strategy chosen by the sync server when merging the sync data.
 # 1. If the sync data accounts for more than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended to select strategy 1.
-#    Setting the parameter to true, which has a greater impact on the write performance of the IoTDB system and occupies less CPU of the machine.
+#    Setting the parameter to true, which has a greater impact on the insert performance of the IoTDB system and occupies less CPU of the machine.
 # 2. If the sync data accounts for less than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended to select strategy 2.
-#    Setting the parameter to false, which has little impact on the write performance of IoTDB system and takes up a large amount of CPU power.
+#    Setting the parameter to false, which has little impact on the insert performance of IoTDB system and takes up a large amount of CPU power.
 update_historical_data_possibility=false
diff --git a/iotdb/iotdb/conf/iotdb-env.sh b/iotdb/iotdb/conf/iotdb-env.sh
index 567aa37..80069cc 100755
--- a/iotdb/iotdb/conf/iotdb-env.sh
+++ b/iotdb/iotdb/conf/iotdb-env.sh
@@ -130,11 +130,11 @@ calculate_heap_sizes
 # Minimum heap size
 #HEAP_NEWSIZE="2G"
 
-LOCAL_JMX=no
+JMX_LOCAL=no
 
 JMX_PORT="31999"
 
-if [ "$LOCAL_JMX" = "yes" ]; then
+if [ "JMX_LOCAL" = "yes" ]; then
 	IOTDB_JMX_OPTS="-Diotdb.jmx.local.port=$JMX_PORT"
 	IOTDB_JMX_OPTS="$IOTDB_JMX_OPTS -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false"
 else
diff --git a/iotdb/iotdb/conf/tsfile-format.properties b/iotdb/iotdb/conf/tsfile-format.properties
index 7148006..24aebf0 100644
--- a/iotdb/iotdb/conf/tsfile-format.properties
+++ b/iotdb/iotdb/conf/tsfile-format.properties
@@ -18,25 +18,33 @@
 #
 
 group_size_in_byte=134217728
+
 # The memory size for each series writer to pack page, default value is 64KB
 page_size_in_byte=65536
-# The maximum number of data points in a page, defalut 1024*1024
+
+# The maximum number of data points in a page, default 1024*1024
 max_number_of_points_in_page=1048576
+
 # Data type configuration
 # Data type for input timestamp, TsFile supports INT32 or INT64
 time_series_data_type=INT64
-# Max length limitation of input string
+
+# Max size limitation of input string
 max_string_length=128
+
 # Floating-point precision
 float_precision=2
+
 # Encoder configuration
 # Encoder of time series, TsFile supports TS_2DIFF, PLAIN and RLE(run-length encoding) and default value is TS_2DIFF
-time_series_encoder=TS_2DIFF
+time_encoder=TS_2DIFF
+
 # Encoder of value series. default value is PLAIN.
 # For int, long data type, TsFile also supports TS_2DIFF and RLE(run-length encoding).
 # For float, double data type, TsFile also supports TS_2DIFF, RLE(run-length encoding) and GORILLA.
 # For text data type, TsFile only supports PLAIN.
 value_encoder=PLAIN
+
 # Compression configuration
 # Data compression method, TsFile supports UNCOMPRESSED or SNAPPY. Default value is UNCOMPRESSED which means no compression
 compressor=UNCOMPRESSED
\ No newline at end of file
diff --git a/iotdb/pom.xml b/iotdb/pom.xml
index 7ff37e9..eba79cc 100644
--- a/iotdb/pom.xml
+++ b/iotdb/pom.xml
@@ -163,6 +163,32 @@
                     <skipITs>${iotdb.it.skip}</skipITs>
                 </configuration>
             </plugin>
+            <!-- for TsFile Checker -->
+            <!--<plugin>-->
+            <!--<groupId>org.apache.maven.plugins</groupId>-->
+            <!--<artifactId>maven-assembly-plugin</artifactId>-->
+            <!--<version>2.5.5</version>-->
+            <!--<configuration>-->
+            <!--<archive>-->
+            <!--<manifest>-->
+            <!--<addClasspath>true</addClasspath>-->
+            <!--<mainClass>org.apache.iotdb.db.tools.TsFileChecker</mainClass>-->
+            <!--</manifest>-->
+            <!--</archive>-->
+            <!--<descriptorRefs>-->
+            <!--<descriptorRef>jar-with-dependencies</descriptorRef>-->
+            <!--</descriptorRefs>-->
+            <!--</configuration>-->
+            <!--<executions>-->
+            <!--<execution>-->
+            <!--<id>make-assembly</id>-->
+            <!--<phase>package</phase>-->
+            <!--<goals>-->
+            <!--<goal>single</goal>-->
+            <!--</goals>-->
+            <!--</execution>-->
+            <!--</executions>-->
+            <!--</plugin>-->
             <!-- Might require this in Eclipse -->
             <!--plugin>
   <groupId>org.codehaus.mojo</groupId>
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
index 439b3d2..3ab4918 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 
 public abstract class BasicAuthorizer implements IAuthorizer, IService {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(BasicAuthorizer.class);
+  private static final Logger logger = LoggerFactory.getLogger(BasicAuthorizer.class);
   private static final Set<Integer> ADMIN_PRIVILEGES;
   private static final String NO_SUCH_ROLE_EXCEPTION = "No such role : %s";
 
@@ -60,7 +60,7 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   protected void init() throws AuthException {
     userManager.reset();
     roleManager.reset();
-    LOGGER.info("Initialization of Authorizer completes");
+    logger.info("Initialization of Authorizer completes");
   }
 
   @Override
@@ -70,20 +70,24 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public boolean createUser(String username, String password) throws AuthException {
-    return userManager.createUser(username, password);
+  public void createUser(String username, String password) throws AuthException {
+    if (!userManager.createUser(username, password)) {
+      throw new AuthException(String.format("User %s already exists", username));
+    }
   }
 
   @Override
-  public boolean deleteUser(String username) throws AuthException {
+  public void deleteUser(String username) throws AuthException {
     if (IoTDBConstant.ADMIN_NAME.equals(username)) {
       throw new AuthException("Default administrator cannot be deleted");
     }
-    return userManager.deleteUser(username);
+    if (!userManager.deleteUser(username)) {
+      throw new AuthException(String.format("User %s does not exist", username));
+    }
   }
 
   @Override
-  public boolean grantPrivilegeToUser(String username, String path, int privilegeId)
+  public void grantPrivilegeToUser(String username, String path, int privilegeId)
       throws AuthException {
     String newPath = path;
     if (IoTDBConstant.ADMIN_NAME.equals(username)) {
@@ -92,11 +96,14 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
       newPath = IoTDBConstant.PATH_ROOT;
     }
-    return userManager.grantPrivilegeToUser(username, newPath, privilegeId);
+    if (!userManager.grantPrivilegeToUser(username, newPath, privilegeId)) {
+      throw new AuthException(String.format(
+          "User %s already has %s on %s", username, PrivilegeType.values()[privilegeId], path));
+    }
   }
 
   @Override
-  public boolean revokePrivilegeFromUser(String username, String path, int privilegeId)
+  public void revokePrivilegeFromUser(String username, String path, int privilegeId)
       throws AuthException {
     if (IoTDBConstant.ADMIN_NAME.equals(username)) {
       throw new AuthException("Invalid operation, administrator must have all privileges");
@@ -105,19 +112,24 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
       p = IoTDBConstant.PATH_ROOT;
     }
-    return userManager.revokePrivilegeFromUser(username, p, privilegeId);
+    if (!userManager.revokePrivilegeFromUser(username, p, privilegeId)) {
+      throw new AuthException(String.format("User %s does not have %s on %s", username,
+          PrivilegeType.values()[privilegeId], path));
+    }
   }
 
   @Override
-  public boolean createRole(String roleName) throws AuthException {
-    return roleManager.createRole(roleName);
+  public void createRole(String roleName) throws AuthException {
+    if (!roleManager.createRole(roleName)) {
+      throw new AuthException(String.format("Role %s already exists", roleName));
+    }
   }
 
   @Override
-  public boolean deleteRole(String roleName) throws AuthException {
+  public void deleteRole(String roleName) throws AuthException {
     boolean success = roleManager.deleteRole(roleName);
     if (!success) {
-      return false;
+      throw new AuthException(String.format("Role %s does not exist", roleName));
     } else {
       // proceed to revoke the role in all users
       List<String> users = userManager.listAllUsers();
@@ -125,37 +137,42 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
         try {
           userManager.revokeRoleFromUser(roleName, user);
         } catch (AuthException e) {
-          LOGGER.warn(
+          logger.warn(
               "Error encountered when revoking a role {} from user {} after deletion, because {}",
               roleName, user, e);
         }
       }
     }
-    return true;
   }
 
   @Override
-  public boolean grantPrivilegeToRole(String roleName, String path, int privilegeId)
+  public void grantPrivilegeToRole(String roleName, String path, int privilegeId)
       throws AuthException {
     String p = path;
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
       p = IoTDBConstant.PATH_ROOT;
     }
-    return roleManager.grantPrivilegeToRole(roleName, p, privilegeId);
+    if(!roleManager.grantPrivilegeToRole(roleName, p, privilegeId)) {
+      throw new AuthException(String.format("Role %s already has %s on %s", roleName,
+          PrivilegeType.values()[privilegeId], path));
+    }
   }
 
   @Override
-  public boolean revokePrivilegeFromRole(String roleName, String path, int privilegeId)
+  public void revokePrivilegeFromRole(String roleName, String path, int privilegeId)
       throws AuthException {
     String p = path;
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
       p = IoTDBConstant.PATH_ROOT;
     }
-    return roleManager.revokePrivilegeFromRole(roleName, p, privilegeId);
+    if (!roleManager.revokePrivilegeFromRole(roleName, p, privilegeId)) {
+      throw new AuthException(String.format("Role %s does not have %s on %s", roleName,
+          PrivilegeType.values()[privilegeId], path));
+    }
   }
 
   @Override
-  public boolean grantRoleToUser(String roleName, String username) throws AuthException {
+  public void grantRoleToUser(String roleName, String username) throws AuthException {
     Role role = roleManager.getRole(roleName);
     if (role == null) {
       throw new AuthException(String.format(NO_SUCH_ROLE_EXCEPTION, roleName));
@@ -166,21 +183,23 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
       role = roleManager.getRole(roleName);
       if (role == null) {
         throw new AuthException(String.format(NO_SUCH_ROLE_EXCEPTION, roleName));
-      } else {
-        return true;
       }
     } else {
-      return false;
+      throw new AuthException(String.format("User %s already has role %s",
+       username, roleName));
     }
   }
 
   @Override
-  public boolean revokeRoleFromUser(String roleName, String username) throws AuthException {
+  public void revokeRoleFromUser(String roleName, String username) throws AuthException {
     Role role = roleManager.getRole(roleName);
     if (role == null) {
       throw new AuthException(String.format(NO_SUCH_ROLE_EXCEPTION, roleName));
     }
-    return userManager.revokeRoleFromUser(roleName, username);
+    if (!userManager.revokeRoleFromUser(roleName, username)) {
+      throw new AuthException(String.format("User %s does not have role %s", username,
+          roleName));
+    }
   }
 
   @Override
@@ -205,8 +224,10 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public boolean updateUserPassword(String username, String newPassword) throws AuthException {
-    return userManager.updateUserPassword(username, newPassword);
+  public void updateUserPassword(String username, String newPassword) throws AuthException {
+    if (!userManager.updateUserPassword(username, newPassword)) {
+      throw new AuthException("password " + newPassword + " is illegal");
+    }
   }
 
   @Override
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/IAuthorizer.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/IAuthorizer.java
index 27d123d..abc690f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/IAuthorizer.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/IAuthorizer.java
@@ -48,22 +48,20 @@ public interface IAuthorizer {
    *            is not null or empty
    * @param password
    *            is not null or empty
-   * @return True if the user is successfully created, false when the user already exists.
    * @throws AuthException
-   *             if the given username or password is illegal.
+   *             if the given username or password is illegal or the user already exists.
    */
-  boolean createUser(String username, String password) throws AuthException;
+  void createUser(String username, String password) throws AuthException;
 
   /**
    * Delete a user.
    *
    * @param username
    *            the username of the user.
-   * @return True if the user is successfully deleted, false if the user does not exists.
    * @throws AuthException
-   *             When attempting to delete the default administrator
+   *             When attempting to delete the default administrator or the user does not exists.
    */
-  boolean deleteUser(String username) throws AuthException;
+  void deleteUser(String username) throws AuthException;
 
   /**
    * Grant a privilege on a seriesPath to a user.
@@ -75,11 +73,10 @@ public interface IAuthorizer {
    *            seriesPath-free privilege, this should be "root".
    * @param privilegeId
    *            An integer that represents a privilege.
-   * @return True if the permission is successfully added, false if the permission already exists.
    * @throws AuthException
-   *             If the user does not exist or the privilege or the seriesPath is illegal.
+   *             If the user does not exist or the privilege or the seriesPath is illegal or the permission already exists.
    */
-  boolean grantPrivilegeToUser(String username, String path, int privilegeId) throws AuthException;
+  void grantPrivilegeToUser(String username, String path, int privilegeId) throws AuthException;
 
   /**
    * Revoke a privilege on seriesPath from a user.
@@ -91,12 +88,11 @@ public interface IAuthorizer {
    *            seriesPath-free privilege, this should be "root".
    * @param privilegeId
    *            An integer that represents a privilege.
-   * @return True if the permission is successfully revoked,
-   *          false if the permission does not exists.
    * @throws AuthException
-   *             If the user does not exist or the privilege or the seriesPath is illegal.
+   *             If the user does not exist or the privilege or the seriesPath is illegal or if
+   *             the permission does not exist.
    */
-  boolean revokePrivilegeFromUser(String username, String path,
+  void revokePrivilegeFromUser(String username, String path,
       int privilegeId) throws AuthException;
 
   /**
@@ -104,20 +100,18 @@ public interface IAuthorizer {
    *
    * @param roleName
    *            the name of the role to be added.
-   * @return True if the role is successfully added, false if the role already exists
-   * @throws AuthException if exception raised when adding the role.
+   * @throws AuthException if exception raised when adding the role or the role already exists.
    */
-  boolean createRole(String roleName) throws AuthException;
+  void createRole(String roleName) throws AuthException;
 
   /**
    * Delete a role.
    *
    * @param roleName
    *            the name of the role tobe deleted.
-   * @return True if the role is successfully deleted, false if the role does not exists.
-   * @throws AuthException if exception raised when deleting the role.
+   * @throws AuthException if exception raised when deleting the role or the role does not exists.
    */
-  boolean deleteRole(String roleName) throws AuthException;
+  void deleteRole(String roleName) throws AuthException;
 
   /**
    * Add a privilege on a seriesPath to a role.
@@ -129,11 +123,11 @@ public interface IAuthorizer {
    *            a seriesPath-free privilege, this should be "root".
    * @param privilegeId
    *            An integer that represents a privilege.
-   * @return True if the privilege is successfully granted, false if the privilege already exists.
    * @throws AuthException
-   *             If the role does not exist or the privilege or the seriesPath is illegal.
+   *             If the role does not exist or the privilege or the seriesPath is illegal or
+   *             the privilege already exists.
    */
-  boolean grantPrivilegeToRole(String roleName, String path, int privilegeId) throws AuthException;
+  void grantPrivilegeToRole(String roleName, String path, int privilegeId) throws AuthException;
 
   /**
    * Remove a privilege on a seriesPath from a role.
@@ -145,11 +139,11 @@ public interface IAuthorizer {
    *            seriesPath-free privilege, this should be "root".
    * @param privilegeId
    *            An integer that represents a privilege.
-   * @return True if the privilege is successfully revoked, false if the privilege does not exists.
    * @throws AuthException
-   *             If the role does not exist or the privilege or the seriesPath is illegal.
+   *             If the role does not exist or the privilege or the seriesPath is illegal or the
+   *             privilege does not exists.
    */
-  boolean revokePrivilegeFromRole(String roleName, String path,
+  void revokePrivilegeFromRole(String roleName, String path,
       int privilegeId) throws AuthException;
 
   /**
@@ -159,11 +153,10 @@ public interface IAuthorizer {
    *            The name of the role to be added.
    * @param username
    *            The name of the user to which the role is added.
-   * @return True if the role is successfully added, false if the role already exists.
    * @throws AuthException
-   *             If either the role or the user does not exist.
+   *             If either the role or the user does not exist or the role already exists.
    */
-  boolean grantRoleToUser(String roleName, String username) throws AuthException;
+  void grantRoleToUser(String roleName, String username) throws AuthException;
 
   /**
    * Revoke a role from a user.
@@ -172,11 +165,10 @@ public interface IAuthorizer {
    *            The name of the role to be removed.
    * @param username
    *            The name of the user from which the role is removed.
-   * @return True if the role is successfully removed, false if the role already exists.
    * @throws AuthException
-   *             If either the role or the user does not exist.
+   *             If either the role or the user does not exist or the role already exists.
    */
-  boolean revokeRoleFromUser(String roleName, String username) throws AuthException;
+  void revokeRoleFromUser(String roleName, String username) throws AuthException;
 
   /**
    * Get the all the privileges of a user on a seriesPath.
@@ -198,11 +190,10 @@ public interface IAuthorizer {
    *            The user whose password is to be modified.
    * @param newPassword
    *            The new password.
-   * @return True if the password is successfully modified, false if the new password is illegal.
    * @throws AuthException
-   *             If the user does not exists.
+   *             If the user does not exists or  the new password is illegal.
    */
-  boolean updateUserPassword(String username, String newPassword) throws AuthException;
+  void updateUserPassword(String username, String newPassword) throws AuthException;
 
   /**
    * Check if the user have the privilege on the seriesPath.
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizer.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizer.java
index e1afe1b..2d9ee21 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizer.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizer.java
@@ -33,8 +33,8 @@ public class LocalFileAuthorizer extends BasicAuthorizer {
   private static Logger logger = LoggerFactory.getLogger(LocalFileAuthorizer.class);
 
   private LocalFileAuthorizer() throws AuthException {
-    super(new LocalFileUserManager(config.getDataDir() + File.separator + "users" + File.separator),
-        new LocalFileRoleManager(config.getDataDir() + File.separator + "roles" + File.separator));
+    super(new LocalFileUserManager(config.getSystemDir() + File.separator + "users"),
+        new LocalFileRoleManager(config.getSystemDir() + File.separator + "roles"));
   }
 
   /**
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessor.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessor.java
index b8f8f64..0adc30c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessor.java
@@ -37,11 +37,11 @@ import org.apache.iotdb.db.utils.IOUtils;
 
 /**
  * This class store each role in a separate sequential file. Role file schema : Int32 role name
- * length Utf-8 role name bytes Int32 seriesPath privilege number n Int32 seriesPath[1] length Utf-8
+ * size Utf-8 role name bytes Int32 seriesPath privilege number n Int32 seriesPath[1] size Utf-8
  * seriesPath[1] bytes Int32 privilege num k1 Int32 privilege[1][1] Int32 privilege[1][2] ... Int32
- * privilege[1][k1] Int32 seriesPath[2] length Utf-8 seriesPath[2] bytes Int32 privilege num yk2
+ * privilege[1][k1] Int32 seriesPath[2] size Utf-8 seriesPath[2] bytes Int32 privilege num yk2
  * Int32 privilege[2][1] Int32 privilege[2][2] ... Int32 privilege[2][k2] ... Int32 seriesPath[n]
- * length Utf-8 seriesPath[n] bytes Int32 privilege num kn Int32 privilege[n][1] Int32
+ * size Utf-8 seriesPath[n] bytes Int32 privilege num kn Int32 privilege[n][1] Int32
  * privilege[n][2] ... Int32 privilege[n][kn]
  */
 public class LocalFileRoleAccessor implements IRoleAccessor {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/user/BasicUserManager.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/user/BasicUserManager.java
index e79b634..d2011e5 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/user/BasicUserManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/user/BasicUserManager.java
@@ -250,7 +250,7 @@ public abstract class BasicUserManager implements IUserManager {
     try {
       User user = getUser(username);
       if (user == null) {
-        throw new AuthException(String.format("No such user %s", username));
+        throw new AuthException(String.format(NO_SUCH_USER_ERROR, username));
       }
       if (!user.hasRole(roleName)) {
         return false;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessor.java b/iotdb/src/main/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessor.java
index b3020ad..e712de2 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessor.java
@@ -39,21 +39,21 @@ import org.slf4j.LoggerFactory;
 
 /**
  * This class loads a user's information from the corresponding file.The user file is a sequential
- * file. User file schema: Int32 username bytes length Utf-8 username bytes Int32 Password bytes
- * length Utf-8 password bytes Int32 seriesPath privilege number n Int32 seriesPath[1] length Utf-8
+ * file. User file schema: Int32 username bytes size Utf-8 username bytes Int32 Password bytes
+ * size Utf-8 password bytes Int32 seriesPath privilege number n Int32 seriesPath[1] size Utf-8
  * seriesPath[1] bytes Int32 privilege num k1 Int32 privilege[1][1] Int32 privilege[1][2] ... Int32
- * privilege[1][k1] Int32 seriesPath[2] length Utf-8 seriesPath[2] bytes Int32 privilege num k2
+ * privilege[1][k1] Int32 seriesPath[2] size Utf-8 seriesPath[2] bytes Int32 privilege num k2
  * Int32 privilege[2][1] Int32 privilege[2][2] ... Int32 privilege[2][k2] ... Int32 seriesPath[n]
- * length Utf-8 seriesPath[n] bytes Int32 privilege num kn Int32 privilege[n][1] Int32
- * privilege[n][2] ... Int32 privilege[n][kn] Int32 user name number m Int32 user name[1] length
- * Utf-8 user name[1] bytes Int32 user name[2] length Utf-8 user name[2] bytes ... Int32 user
- * name[m] length Utf-8 user name[m] bytes
+ * size Utf-8 seriesPath[n] bytes Int32 privilege num kn Int32 privilege[n][1] Int32
+ * privilege[n][2] ... Int32 privilege[n][kn] Int32 user name number m Int32 user name[1] size
+ * Utf-8 user name[1] bytes Int32 user name[2] size Utf-8 user name[2] bytes ... Int32 user
+ * name[m] size Utf-8 user name[m] bytes
  */
 public class LocalFileUserAccessor implements IUserAccessor {
 
   private static final String TEMP_SUFFIX = ".temp";
   private static final String STRING_ENCODING = "utf-8";
-  private static final Logger LOGGER = LoggerFactory.getLogger(LocalFileUserAccessor.class);
+  private static final Logger logger = LoggerFactory.getLogger(LocalFileUserAccessor.class);
 
   private String userDirPath;
   /**
@@ -83,7 +83,7 @@ public class LocalFileUserAccessor implements IUserAccessor {
           userDirPath + File.separator + username + IoTDBConstant.PROFILE_SUFFIX + TEMP_SUFFIX);
       if (newProfile.exists() && newProfile.isFile()) {
         if(!newProfile.renameTo(userProfile)) {
-          LOGGER.error("New profile renaming not succeed.");
+          logger.error("New profile renaming not succeed.");
         }
         userProfile = newProfile;
       } else {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/concurrent/IoTDBDefaultThreadExceptionHandler.java b/iotdb/src/main/java/org/apache/iotdb/db/concurrent/IoTDBDefaultThreadExceptionHandler.java
index 0623d16..05c44e6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/concurrent/IoTDBDefaultThreadExceptionHandler.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/concurrent/IoTDBDefaultThreadExceptionHandler.java
@@ -23,12 +23,12 @@ import org.slf4j.LoggerFactory;
 
 public class IoTDBDefaultThreadExceptionHandler implements Thread.UncaughtExceptionHandler {
 
-  private static final Logger LOGGER = LoggerFactory
+  private static final Logger logger = LoggerFactory
       .getLogger(IoTDBDefaultThreadExceptionHandler.class);
 
   @Override
   public void uncaughtException(Thread t, Throwable e) {
-    LOGGER.error("Exception in thread {}-{}", t.getName(), t.getId(), e);
+    logger.error("Exception in thread {}-{}", t.getName(), t.getId(), e);
   }
 
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java b/iotdb/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
index a1c3e44..132df0d 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
@@ -32,6 +32,7 @@ public enum ThreadName {
   FORCE_FLUSH_ALL_POLICY("IoTDB-ForceFlushAllPolicy-Thread"),
   STAT_MONITOR("StatMonitor-ServerServiceImpl"),
   FLUSH_SERVICE("Flush-ServerServiceImpl"),
+  FLUSH_SUB_TASK_SERVICE("Flush-SubTask-ServerServiceImpl"),
   WAL_DAEMON("IoTDB-MultiFileLogNodeManager-Sync-Thread"),
   WAL_FORCE_DAEMON("IoTDB-MultiFileLogNodeManager-Force-Thread"),
   INDEX_SERVICE("Index-ServerServiceImpl"),
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 4bf5ac8..82dab68 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -18,10 +18,7 @@
  */
 package org.apache.iotdb.db.conf;
 
-import java.io.File;
 import java.time.ZoneId;
-import java.util.ArrayList;
-import java.util.List;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.service.TSServiceImpl;
 import org.slf4j.Logger;
@@ -29,14 +26,11 @@ import org.slf4j.LoggerFactory;
 
 public class IoTDBConfig {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConfig.class);
+  private static final Logger logger = LoggerFactory.getLogger(IoTDBConfig.class);
   public static final String CONFIG_NAME = "iotdb-engine.properties";
-  public static final String DEFAULT_DATA_DIR = "data";
-  public static final String DEFAULT_SYS_DIR = "system";
-  public static final String DEFAULT_TSFILE_DIR = "settled";
-  public static final String MULT_DIR_STRATEGY_PREFIX =
+  private static final String MULTI_DIR_STRATEGY_PREFIX =
       "org.apache.iotdb.db.conf.directories.strategy.";
-  public static final String DEFAULT_MULT_DIR_STRATEGY = "MaxDiskUsableSpaceFirstStrategy";
+  private static final String DEFAULT_MULTI_DIR_STRATEGY = "MaxDiskUsableSpaceFirstStrategy";
 
   private String rpcAddress = "0.0.0.0";
   /**
@@ -47,7 +41,7 @@ public class IoTDBConfig {
   /**
    * Is the write ahead log enable.
    */
-  private boolean enableWal = false;
+  private boolean enableWal = true;
 
   /**
    * When a certain amount of write ahead logs is reached, they will be flushed to the disk. It is
@@ -56,88 +50,55 @@ public class IoTDBConfig {
   private int flushWalThreshold = 10000;
 
   /**
-   * The cycle when write ahead logs are periodically refreshed to disk(in milliseconds). It is
-   * possible to lose at most flush_wal_period_in_ms ms operations.
-   */
-  private long flushWalPeriodInMs = 10000;
-
-  /**
    * The cycle when write ahead log is periodically forced to be written to disk(in milliseconds) If
-   * set this parameter to 0 it means call outputStream.force(true) after every each write
+   * set this parameter to 0 it means call outputStream.force(true) after every each insert
    */
   private long forceWalPeriodInMs = 10;
 
   /**
-   * Data directory.
+   * Size of log buffer in each log node(in byte). If WAL is enabled and the size of a insert plan
+   * is smaller than this parameter, then the insert plan will be rejected by WAL.
    */
-  private String dataDir = null;
+  private int walBufferSize = 16 * 1024 * 1024;
 
   /**
-   * System directory.
-   */
-  private String sysDir = null;
-
-  /**
-   * Wal directory.
+   * system base dir, stores all system metadata and wal
    */
-  private String walDir = null;
+  private String baseDir = "data";
 
   /**
-   * Data directory of Overflow data.
+   * System directory, including version file for each storage group and metadata
    */
-  private String overflowDataDir = "overflow";
+  private String systemDir = "data/system";
 
   /**
-   * Data directory of fileNode data.
+   * Data directory of data. It can be settled as dataDirs = {"data1", "data2", "data3"};
    */
-  private String fileNodeDir = "info";
-
-  /**
-   * Data directory of bufferWrite data.
-   * It can be settled as bufferWriteDirs = {"settled1", "settled2", "settled3"};
-   */
-  private String[] bufferWriteDirs = {DEFAULT_TSFILE_DIR};
+  private String[] dataDirs = {"data/data"};
 
   /**
    * Strategy of multiple directories.
    */
-  private String multDirStrategyClassName = null;
-
-  /**
-   * Data directory of metadata data.
-   */
-  private String metadataDir = "schema";
+  private String multiDirStrategyClassName = null;
 
   /**
-   * Data directory of derby data.
-   */
-  private String derbyHome = "derby";
-
-  /**
-   * Data directory of Write ahead log folder.
+   * Wal directory.
    */
-  private String walFolder = "wal";
+  private String walFolder = "data/wal";
 
   /**
    * Data directory for index files (KV-match indexes).
    */
-  private String indexFileDir = "index";
+  private String indexFileDir = "data/index";
 
-  /**
-   * Temporary directory for temporary files of read (External Sort). TODO: unused field
-   */
-  private String readTmpFileDir = "readTmp";
 
-  /**
-   * The maximum concurrent thread number for merging overflow. When the value <=0 or > CPU core
-   * number, use the CPU core number.
-   */
-  private int mergeConcurrentThreads = Runtime.getRuntime().availableProcessors();
+  private int memtableNumber = 20;
 
   /**
-   * Maximum number of folders open at the same time.
+   * The maximum concurrent thread number for merging. When the value <=0 or > CPU core number, use
+   * the CPU core number.
    */
-  private int maxOpenFolder = 100;
+  private int mergeConcurrentThreads = Runtime.getRuntime().availableProcessors();
 
   /**
    * The amount of data that is read every time when IoTDB merges data.
@@ -145,82 +106,17 @@ public class IoTDBConfig {
   private int fetchSize = 10000;
 
   /**
-   * the maximum number of writing instances existing in same time.
-   */
-  @Deprecated
-  private int writeInstanceThreshold = 5;
-
-  /**
-   * The period time of flushing data from memory to file. . The unit is second.
-   */
-  private long periodTimeForFlush = 3600;
-
-  /**
-   * The period time for merge overflow data with tsfile data. The unit is second.
-   */
-  private long periodTimeForMerge = 7200;
-
-  /**
-   * When set true, start timed flush and merge service. Else, stop timed flush and merge service.
-   * The default value is true. TODO: 'timed' better explains this than 'timing'.
-   */
-  private boolean enableTimingCloseAndMerge = true;
-
-  /**
    * How many threads can concurrently flush. When <= 0, use CPU core number.
    */
   private int concurrentFlushThread = Runtime.getRuntime().availableProcessors();
 
   private ZoneId zoneID = ZoneId.systemDefault();
+
   /**
-   * BufferWriteProcessor and OverflowProcessor will immediately flush if this threshold is
-   * reached.
-   */
-  private long memThresholdWarning = (long) (0.5 * Runtime.getRuntime().maxMemory());
-  /**
-   * No more insert is allowed if this threshold is reached.
-   */
-  private long memThresholdDangerous = (long) (0.6 * Runtime.getRuntime().maxMemory());
-  /**
-   * MemMonitorThread will check every such interval(in ms). If memThresholdWarning is reached,
-   * MemMonitorThread will inform FileNodeManager to flush.
-   */
-  private long memMonitorInterval = 1000;
-  /**
-   * Decide how to control memory usage of inserting data. 0 is RecordMemController, which sums the
-   * size of each record (tuple). 1 is JVMMemController, which uses the JVM heap memory as the
-   * memory usage indicator.
-   */
-  private int memControllerType = 1;
-  /**
-   * When a bufferwrite's metadata size (in byte) exceed this, the bufferwrite is forced closed.
-   */
-  private long bufferwriteMetaSizeThreshold = 200 * 1024 * 1024L;
-  /**
-   * When a bufferwrite's file size (in byte) exceed this, the bufferwrite is forced closed.
-   */
-  private long bufferwriteFileSizeThreshold = 2 * 1024 * 1024 * 1024L;
-  /**
-   * When a overflow's metadata size (in byte) exceed this, the overflow is forced closed.
-   */
-  private long overflowMetaSizeThreshold = 20 * 1024 * 1024L;
-  /**
-   * When a overflow's file size (in byte) exceed this, the overflow is forced closed.
-   */
-  private long overflowFileSizeThreshold = 200 * 1024 * 1024L;
-  /**
-   * If set false, MemMonitorThread and MemStatisticThread will not be created.
-   */
-  private boolean enableMemMonitor = false;
-  /**
-   * When set to true, small flushes will be triggered periodically even if the memory threshold is
-   * not exceeded.
-   */
-  private boolean enableSmallFlush = false;
-  /**
-   * The interval of small flush in ms.
+   * When a TsFile's file size (in byte) exceed this, the TsFile is forced closed.
    */
-  private long smallFlushInterval = 60L * 1000;
+  private long tsFileSizeThreshold = 512 * 1024 * 1024L;
+
   /**
    * The statMonitor writes statistics info into IoTDB every backLoopPeriodSec secs. The default
    * value is 5s.
@@ -239,20 +135,12 @@ public class IoTDBConfig {
    * 600s.
    */
   private int statMonitorRetainIntervalSec = 60 * 10;
-  /**
-   * Threshold for external sort. When using multi-line merging sort, if the count of lines exceed
-   * {@code externalSortThreshold}, it will trigger external sort.
-   */
-  private int externalSortThreshold = 50;
+
   /**
    * Cache size of {@code checkAndGetDataTypeCache} in {@link MManager}.
    */
   private int mManagerCacheSize = 400000;
-  /**
-   * The maximum size of a single log in byte. If a log exceeds this size, it cannot be written to
-   * the WAL file and an exception is thrown.
-   */
-  private int maxLogEntrySize = 4 * 1024 * 1024;
+
   /**
    * Is this IoTDB instance a receiver of sync or not.
    */
@@ -261,7 +149,7 @@ public class IoTDBConfig {
    * If this IoTDB instance is a receiver of sync, set the server port.
    */
   private int syncServerPort = 5555;
-  /*
+  /**
    * Set the language version when loading file including error information, default value is "EN"
    */
   private String languageVersion = "EN";
@@ -271,6 +159,7 @@ public class IoTDBConfig {
    * exactly, choose "false".
    */
   private boolean updateHistoricalDataPossibility = false;
+
   private String ipWhiteList = "0.0.0.0/0";
   /**
    * Examining period of cache file reader : 100 seconds.
@@ -282,6 +171,11 @@ public class IoTDBConfig {
    */
   private String rpcImplClassName = TSServiceImpl.class.getName();
 
+  /**
+   * whether use chunkBufferPool.
+   */
+  private boolean chunkBufferPoolEnable = false;
+
   public IoTDBConfig() {
     // empty constructor
   }
@@ -290,143 +184,36 @@ public class IoTDBConfig {
     return zoneID;
   }
 
-  public String getZoneIDString() {
-    return getZoneID().toString();
-  }
-
-  public void updatePath() {
+  void updatePath() {
     confirmMultiDirStrategy();
-
-    preUpdatePath();
-
-    // update the paths of subdirectories in the dataDir
-    if (getDataDir().length() > 0 && !getDataDir().endsWith(File.separator)) {
-      setDataDir(getDataDir() + File.separatorChar);
-    }
-    setOverflowDataDir(getDataDir() + getOverflowDataDir());
-
-    if (getBufferWriteDirs() == null || getBufferWriteDirs().length == 0) {
-      setBufferWriteDirs(new String[]{DEFAULT_TSFILE_DIR});
-    }
-    for (int i = 0; i < getBufferWriteDirs().length; i++) {
-      if (new File(getBufferWriteDirs()[i]).isAbsolute()) {
-        continue;
-      }
-
-      getBufferWriteDirs()[i] = getDataDir() + getBufferWriteDirs()[i];
-    }
-
-    // update the paths of subdirectories in the sysDir
-    if (getSysDir().length() > 0 && !getSysDir().endsWith(File.separator)) {
-      setSysDir(getSysDir() + File.separatorChar);
-    }
-    setFileNodeDir(getSysDir() + getFileNodeDir());
-    setMetadataDir(getSysDir() + getMetadataDir());
-
-    // update the paths of subdirectories in the walDir
-    if (getWalDir().length() > 0 && !getWalDir().endsWith(File.separator)) {
-      setWalDir(getWalDir() + File.separatorChar);
-    }
-    setWalFolder(getWalDir() + getWalFolder());
-
-    setDerbyHome(getSysDir() + getDerbyHome());
-    setIndexFileDir(getDataDir() + getIndexFileDir());
-  }
-
-  /*
-   * First, if dataDir is null, dataDir will be assigned the default
-   * value(i.e.,"data"+File.separatorChar+"data".
-   * Then, if dataDir is absolute, leave dataDir as it is. If dataDir is relative, dataDir
-   * will be converted to the complete version using non-empty %IOTDB_HOME%. e.g.
-   * for windows platform, | IOTDB_HOME | dataDir before | dataDir
-   * after | |-----------------|--------------------|---------------------------| |
-   * D:\\iotdb\iotdb | null |
-   * D:\\iotdb\iotdb\data\data | | D:\\iotdb\iotdb | dataDir | D:\\iotdb\iotdb\dataDir |
-   * | D:\\iotdb\iotdb |
-   * C:\\dataDir | C:\\dataDir | | D:\\iotdb\iotdb | "" | D:\\iotdb\iotdb\ |
-   *
-   * First, if sysDir is null, sysDir will be assigned the default
-   * value(i.e.,"data"+File.separatorChar+"system".
-   * Then, if sysDir is absolute, leave sysDir as it is. If sysDir is relative,
-   * sysDir will be converted to the complete version using non-empty %IOTDB_HOME%.
-   * e.g. for windows platform, | IOTDB_HOME | sysDir before | sysDir
-   * after | |-----------------|--------------------|-----------------------------|
-   * | D:\\iotdb\iotdb | null |D:\\iotdb\iotdb\data\system | | D:\\iotdb\iotdb | sysDir
-   * | D:\\iotdb\iotdb\sysDir | | D:\\iotdb\iotdb |
-   * C:\\sysDir | C:\\sysDir | | D:\\iotdb\iotdb | "" | D:\\iotdb\iotdb\ |
-   *
-   * First, if walDir is null, walDir will be assigned the default
-   * value(i.e.,"data"+File.separatorChar+"data". Then,
-   * if walDir is absolute, leave walDir as it is. If walDir is relative,
-   * walDir will be converted to the complete
-   * version using non-empty %IOTDB_HOME%. e.g. for windows platform,
-   * | IOTDB_HOME | walDir before | walDir after |
-   * |-----------------|--------------------|-----------------------------|
-   * | D:\\iotdb\iotdb | null |
-   * D:\\iotdb\iotdb\data\wal | | D:\\iotdb\iotdb | walDir | D:\\iotdb\iotdb\walDir |
-   * | D:\\iotdb\iotdb | C:\\walDir |
-   * C:\\walDir | | D:\\iotdb\iotdb | "" | D:\\iotdb\iotdb\ |
-   *
-   */
-
-  private void preUpdatePath() {
-    if (getDataDir() == null) {
-      setDataDir(DEFAULT_DATA_DIR + File.separatorChar + DEFAULT_DATA_DIR);
-    }
-    if (getSysDir() == null) {
-      setSysDir(DEFAULT_DATA_DIR + File.separatorChar + DEFAULT_SYS_DIR);
-    }
-    if (getWalDir() == null) {
-      setWalDir(DEFAULT_DATA_DIR);
-    }
-
-    List<String> dirs = new ArrayList<>();
-    dirs.add(getDataDir());
-    dirs.add(getSysDir());
-    dirs.add(getWalDir());
-    String homeDir = System.getProperty(IoTDBConstant.IOTDB_HOME, null);
-    for (int i = 0; i < 3; i++) {
-      String dir = dirs.get(i);
-      if (!new File(dir).isAbsolute() && homeDir != null && homeDir.length() > 0) {
-        if (!homeDir.endsWith(File.separator)) {
-          dir = homeDir + File.separatorChar + dir;
-        } else {
-          dir = homeDir + dir;
-        }
-        dirs.set(i, dir);
-      }
-    }
-    setDataDir(dirs.get(0));
-    setSysDir(dirs.get(1));
-    setWalDir(dirs.get(2));
   }
 
   private void confirmMultiDirStrategy() {
-    if (getMultDirStrategyClassName() == null) {
-      setMultDirStrategyClassName(DEFAULT_MULT_DIR_STRATEGY);
+    if (getMultiDirStrategyClassName() == null) {
+      multiDirStrategyClassName = DEFAULT_MULTI_DIR_STRATEGY;
     }
-    if (!getMultDirStrategyClassName().contains(".")) {
-      setMultDirStrategyClassName(MULT_DIR_STRATEGY_PREFIX + getMultDirStrategyClassName());
+    if (!getMultiDirStrategyClassName().contains(".")) {
+      multiDirStrategyClassName = MULTI_DIR_STRATEGY_PREFIX + multiDirStrategyClassName;
     }
 
     try {
-      Class.forName(getMultDirStrategyClassName());
+      Class.forName(multiDirStrategyClassName);
     } catch (ClassNotFoundException e) {
-      LOGGER.warn("Cannot find given directory strategy {}, using the default value",
-          getMultDirStrategyClassName(), e);
-      setMultDirStrategyClassName(MULT_DIR_STRATEGY_PREFIX + DEFAULT_MULT_DIR_STRATEGY);
+      logger.warn("Cannot find given directory strategy {}, using the default value",
+          getMultiDirStrategyClassName(), e);
+      setMultiDirStrategyClassName(MULTI_DIR_STRATEGY_PREFIX + DEFAULT_MULTI_DIR_STRATEGY);
     }
   }
 
-  public String[] getBufferWriteDirs() {
-    return bufferWriteDirs;
+  public String[] getDataDirs() {
+    return dataDirs;
   }
 
   public String getRpcAddress() {
     return rpcAddress;
   }
 
-  public void setRpcAddress(String rpcAddress) {
+  void setRpcAddress(String rpcAddress) {
     this.rpcAddress = rpcAddress;
   }
 
@@ -434,7 +221,7 @@ public class IoTDBConfig {
     return rpcPort;
   }
 
-  public void setRpcPort(int rpcPort) {
+  void setRpcPort(int rpcPort) {
     this.rpcPort = rpcPort;
   }
 
@@ -454,14 +241,6 @@ public class IoTDBConfig {
     this.flushWalThreshold = flushWalThreshold;
   }
 
-  public long getFlushWalPeriodInMs() {
-    return flushWalPeriodInMs;
-  }
-
-  public void setFlushWalPeriodInMs(long flushWalPeriodInMs) {
-    this.flushWalPeriodInMs = flushWalPeriodInMs;
-  }
-
   public long getForceWalPeriodInMs() {
     return forceWalPeriodInMs;
   }
@@ -470,259 +249,91 @@ public class IoTDBConfig {
     this.forceWalPeriodInMs = forceWalPeriodInMs;
   }
 
-  public String getDataDir() {
-    return dataDir;
-  }
-
-  public void setDataDir(String dataDir) {
-    this.dataDir = dataDir;
-  }
-
-  public String getSysDir() {
-    return sysDir;
-  }
-
-  public void setSysDir(String sysDir) {
-    this.sysDir = sysDir;
-  }
-
-  public String getWalDir() {
-    return walDir;
-  }
-
-  public void setWalDir(String walDir) {
-    this.walDir = walDir;
-  }
-
-  public String getOverflowDataDir() {
-    return overflowDataDir;
-  }
-
-  public void setOverflowDataDir(String overflowDataDir) {
-    this.overflowDataDir = overflowDataDir;
-  }
-
-  public String getFileNodeDir() {
-    return fileNodeDir;
-  }
-
-  public void setFileNodeDir(String fileNodeDir) {
-    this.fileNodeDir = fileNodeDir;
-  }
-
-  public void setBufferWriteDirs(String[] bufferWriteDirs) {
-    this.bufferWriteDirs = bufferWriteDirs;
-  }
-
-  public String getMultDirStrategyClassName() {
-    return multDirStrategyClassName;
+  public String getSystemDir() {
+    return systemDir;
   }
 
-  public void setMultDirStrategyClassName(String multDirStrategyClassName) {
-    this.multDirStrategyClassName = multDirStrategyClassName;
+  void setSystemDir(String systemDir) {
+    this.systemDir = systemDir;
   }
 
-  public String getMetadataDir() {
-    return metadataDir;
+  public String getWalFolder() {
+    return walFolder;
   }
 
-  public void setMetadataDir(String metadataDir) {
-    this.metadataDir = metadataDir;
+  void setWalFolder(String walFolder) {
+    this.walFolder = walFolder;
   }
 
-  public String getDerbyHome() {
-    return derbyHome;
+  void setDataDirs(String[] dataDirs) {
+    this.dataDirs = dataDirs;
   }
 
-  public void setDerbyHome(String derbyHome) {
-    this.derbyHome = derbyHome;
+  public String getMultiDirStrategyClassName() {
+    return multiDirStrategyClassName;
   }
 
-  public String getWalFolder() {
-    return walFolder;
-  }
-
-  public void setWalFolder(String walFolder) {
-    this.walFolder = walFolder;
+  void setMultiDirStrategyClassName(String multiDirStrategyClassName) {
+    this.multiDirStrategyClassName = multiDirStrategyClassName;
   }
 
   public String getIndexFileDir() {
     return indexFileDir;
   }
 
-  public void setIndexFileDir(String indexFileDir) {
+  private void setIndexFileDir(String indexFileDir) {
     this.indexFileDir = indexFileDir;
   }
 
-  public String getReadTmpFileDir() {
-    return readTmpFileDir;
-  }
-
-  public void setReadTmpFileDir(String readTmpFileDir) {
-    this.readTmpFileDir = readTmpFileDir;
-  }
-
   public int getMergeConcurrentThreads() {
     return mergeConcurrentThreads;
   }
 
-  public void setMergeConcurrentThreads(int mergeConcurrentThreads) {
+  void setMergeConcurrentThreads(int mergeConcurrentThreads) {
     this.mergeConcurrentThreads = mergeConcurrentThreads;
   }
 
-  public int getMaxOpenFolder() {
-    return maxOpenFolder;
-  }
-
-  public void setMaxOpenFolder(int maxOpenFolder) {
-    this.maxOpenFolder = maxOpenFolder;
-  }
-
   public int getFetchSize() {
     return fetchSize;
   }
 
-  public void setFetchSize(int fetchSize) {
+  void setFetchSize(int fetchSize) {
     this.fetchSize = fetchSize;
   }
 
-  public int getWriteInstanceThreshold() {
-    return writeInstanceThreshold;
-  }
-
-  public void setWriteInstanceThreshold(int writeInstanceThreshold) {
-    this.writeInstanceThreshold = writeInstanceThreshold;
-  }
-
-  public long getPeriodTimeForFlush() {
-    return periodTimeForFlush;
-  }
-
-  public void setPeriodTimeForFlush(long periodTimeForFlush) {
-    this.periodTimeForFlush = periodTimeForFlush;
-  }
-
-  public long getPeriodTimeForMerge() {
-    return periodTimeForMerge;
-  }
-
-  public void setPeriodTimeForMerge(long periodTimeForMerge) {
-    this.periodTimeForMerge = periodTimeForMerge;
+  public int getMemtableNumber() {
+    return memtableNumber;
   }
 
-  public boolean isEnableTimingCloseAndMerge() {
-    return enableTimingCloseAndMerge;
-  }
-
-  public void setEnableTimingCloseAndMerge(boolean enableTimingCloseAndMerge) {
-    this.enableTimingCloseAndMerge = enableTimingCloseAndMerge;
+  void setMemtableNumber(int memtableNumber) {
+    this.memtableNumber = memtableNumber;
   }
 
   public int getConcurrentFlushThread() {
     return concurrentFlushThread;
   }
 
-  public void setConcurrentFlushThread(int concurrentFlushThread) {
+  void setConcurrentFlushThread(int concurrentFlushThread) {
     this.concurrentFlushThread = concurrentFlushThread;
   }
 
-  public void setZoneID(ZoneId zoneID) {
+  void setZoneID(ZoneId zoneID) {
     this.zoneID = zoneID;
   }
 
-  public long getMemThresholdWarning() {
-    return memThresholdWarning;
-  }
-
-  public void setMemThresholdWarning(long memThresholdWarning) {
-    this.memThresholdWarning = memThresholdWarning;
-  }
-
-  public long getMemThresholdDangerous() {
-    return memThresholdDangerous;
-  }
-
-  public void setMemThresholdDangerous(long memThresholdDangerous) {
-    this.memThresholdDangerous = memThresholdDangerous;
-  }
-
-  public long getMemMonitorInterval() {
-    return memMonitorInterval;
-  }
-
-  public void setMemMonitorInterval(long memMonitorInterval) {
-    this.memMonitorInterval = memMonitorInterval;
+  public long getTsFileSizeThreshold() {
+    return tsFileSizeThreshold;
   }
 
-  public int getMemControllerType() {
-    return memControllerType;
-  }
-
-  public void setMemControllerType(int memControllerType) {
-    this.memControllerType = memControllerType;
-  }
-
-  public long getBufferwriteMetaSizeThreshold() {
-    return bufferwriteMetaSizeThreshold;
-  }
-
-  public void setBufferwriteMetaSizeThreshold(long bufferwriteMetaSizeThreshold) {
-    this.bufferwriteMetaSizeThreshold = bufferwriteMetaSizeThreshold;
-  }
-
-  public long getBufferwriteFileSizeThreshold() {
-    return bufferwriteFileSizeThreshold;
-  }
-
-  public void setBufferwriteFileSizeThreshold(long bufferwriteFileSizeThreshold) {
-    this.bufferwriteFileSizeThreshold = bufferwriteFileSizeThreshold;
-  }
-
-  public long getOverflowMetaSizeThreshold() {
-    return overflowMetaSizeThreshold;
-  }
-
-  public void setOverflowMetaSizeThreshold(long overflowMetaSizeThreshold) {
-    this.overflowMetaSizeThreshold = overflowMetaSizeThreshold;
-  }
-
-  public long getOverflowFileSizeThreshold() {
-    return overflowFileSizeThreshold;
-  }
-
-  public void setOverflowFileSizeThreshold(long overflowFileSizeThreshold) {
-    this.overflowFileSizeThreshold = overflowFileSizeThreshold;
-  }
-
-  public boolean isEnableMemMonitor() {
-    return enableMemMonitor;
-  }
-
-  public void setEnableMemMonitor(boolean enableMemMonitor) {
-    this.enableMemMonitor = enableMemMonitor;
-  }
-
-  public boolean isEnableSmallFlush() {
-    return enableSmallFlush;
-  }
-
-  public void setEnableSmallFlush(boolean enableSmallFlush) {
-    this.enableSmallFlush = enableSmallFlush;
-  }
-
-  public long getSmallFlushInterval() {
-    return smallFlushInterval;
-  }
-
-  public void setSmallFlushInterval(long smallFlushInterval) {
-    this.smallFlushInterval = smallFlushInterval;
+  void setTsFileSizeThreshold(long tsFileSizeThreshold) {
+    this.tsFileSizeThreshold = tsFileSizeThreshold;
   }
 
   public int getBackLoopPeriodSec() {
     return backLoopPeriodSec;
   }
 
-  public void setBackLoopPeriodSec(int backLoopPeriodSec) {
+  void setBackLoopPeriodSec(int backLoopPeriodSec) {
     this.backLoopPeriodSec = backLoopPeriodSec;
   }
 
@@ -738,7 +349,7 @@ public class IoTDBConfig {
     return statMonitorDetectFreqSec;
   }
 
-  public void setStatMonitorDetectFreqSec(int statMonitorDetectFreqSec) {
+  void setStatMonitorDetectFreqSec(int statMonitorDetectFreqSec) {
     this.statMonitorDetectFreqSec = statMonitorDetectFreqSec;
   }
 
@@ -746,39 +357,23 @@ public class IoTDBConfig {
     return statMonitorRetainIntervalSec;
   }
 
-  public void setStatMonitorRetainIntervalSec(int statMonitorRetainIntervalSec) {
+  void setStatMonitorRetainIntervalSec(int statMonitorRetainIntervalSec) {
     this.statMonitorRetainIntervalSec = statMonitorRetainIntervalSec;
   }
 
-  public int getExternalSortThreshold() {
-    return externalSortThreshold;
-  }
-
-  public void setExternalSortThreshold(int externalSortThreshold) {
-    this.externalSortThreshold = externalSortThreshold;
-  }
-
   public int getmManagerCacheSize() {
     return mManagerCacheSize;
   }
 
-  public void setmManagerCacheSize(int mManagerCacheSize) {
+  void setmManagerCacheSize(int mManagerCacheSize) {
     this.mManagerCacheSize = mManagerCacheSize;
   }
 
-  public int getMaxLogEntrySize() {
-    return maxLogEntrySize;
-  }
-
-  public void setMaxLogEntrySize(int maxLogEntrySize) {
-    this.maxLogEntrySize = maxLogEntrySize;
-  }
-
   public boolean isSyncEnable() {
     return isSyncEnable;
   }
 
-  public void setSyncEnable(boolean syncEnable) {
+  void setSyncEnable(boolean syncEnable) {
     isSyncEnable = syncEnable;
   }
 
@@ -786,7 +381,7 @@ public class IoTDBConfig {
     return syncServerPort;
   }
 
-  public void setSyncServerPort(int syncServerPort) {
+  void setSyncServerPort(int syncServerPort) {
     this.syncServerPort = syncServerPort;
   }
 
@@ -794,7 +389,7 @@ public class IoTDBConfig {
     return languageVersion;
   }
 
-  public void setLanguageVersion(String languageVersion) {
+  void setLanguageVersion(String languageVersion) {
     this.languageVersion = languageVersion;
   }
 
@@ -802,10 +397,18 @@ public class IoTDBConfig {
     return updateHistoricalDataPossibility;
   }
 
-  public void setUpdateHistoricalDataPossibility(boolean updateHistoricalDataPossibility) {
+  void setUpdateHistoricalDataPossibility(boolean updateHistoricalDataPossibility) {
     this.updateHistoricalDataPossibility = updateHistoricalDataPossibility;
   }
 
+  public String getBaseDir() {
+    return baseDir;
+  }
+
+  public void setBaseDir(String baseDir) {
+    this.baseDir = baseDir;
+  }
+
   public String getIpWhiteList() {
     return ipWhiteList;
   }
@@ -829,4 +432,20 @@ public class IoTDBConfig {
   public void setRpcImplClassName(String rpcImplClassName) {
     this.rpcImplClassName = rpcImplClassName;
   }
+
+  public int getWalBufferSize() {
+    return walBufferSize;
+  }
+
+  void setWalBufferSize(int walBufferSize) {
+    this.walBufferSize = walBufferSize;
+  }
+
+  public boolean isChunkBufferPoolEnable() {
+    return chunkBufferPoolEnable;
+  }
+
+  void setChunkBufferPoolEnable(boolean chunkBufferPoolEnable) {
+    this.chunkBufferPoolEnable = chunkBufferPoolEnable;
+  }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index e2775cd..5045194 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -40,13 +40,10 @@ public class IoTDBConstant {
   public static final long MB = 1024 * 1024L;
   public static final long KB = 1024L;
 
-  public static final long MEM_THRESHOLD_WARNING_DEFAULT = 8 * GB;
-  public static final long MEM_THRESHOLD_DANGEROUS_DEFAULT = 16 * GB;
-
   public static final String IOTDB_HOME = "IOTDB_HOME";
 
-  public static final String BUFFERWRITE_LOG_NODE_SUFFIX = "-bufferwrite";
-  public static final String OVERFLOW_LOG_NODE_SUFFIX = "-overflow";
+  public static final String SEQFILE_LOG_NODE_SUFFIX = "-seq";
+  public static final String UNSEQFILE_LOG_NODE_SUFFIX = "-unseq";
 
   public static final String PATH_ROOT = "root";
   public static final char PATH_SEPARATOR = '.';
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 57eedfe..57e941e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -25,13 +25,13 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.time.ZoneId;
 import java.util.Properties;
-import org.apache.iotdb.db.engine.memcontrol.BasicMemController.ControllerType;
+import org.apache.iotdb.db.utils.FilePathUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class IoTDBDescriptor {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBDescriptor.class);
+  private static final Logger logger = LoggerFactory.getLogger(IoTDBDescriptor.class);
   private IoTDBConfig conf = new IoTDBConfig();
 
   private IoTDBDescriptor() {
@@ -46,67 +46,75 @@ public class IoTDBDescriptor {
     return conf;
   }
 
-  /**
-   * load an property file and set TsfileDBConfig variables.
-   */
-  private void loadProps() {
-    InputStream inputStream;
-
+  private String getPropsUrl() {
     String url = System.getProperty(IoTDBConstant.IOTDB_CONF, null);
     if (url == null) {
       url = System.getProperty(IoTDBConstant.IOTDB_HOME, null);
       if (url != null) {
         url = url + File.separatorChar + "conf" + File.separatorChar + IoTDBConfig.CONFIG_NAME;
       } else {
-        LOGGER.warn(
+        logger.warn(
             "Cannot find IOTDB_HOME or IOTDB_CONF environment variable when loading "
                 + "config file {}, use default configuration",
             IoTDBConfig.CONFIG_NAME);
         // update all data seriesPath
         conf.updatePath();
-        return;
+        return null;
       }
     } else {
       url += (File.separatorChar + IoTDBConfig.CONFIG_NAME);
     }
+    return url;
+  }
+
+  /**
+   * load an property file and set TsfileDBConfig variables.
+   */
+  private void loadProps() {
+    InputStream inputStream;
+
+    String url = getPropsUrl();
+    if (url == null) {
+      return;
+    }
 
     try {
       inputStream = new FileInputStream(new File(url));
     } catch (FileNotFoundException e) {
-      LOGGER.warn("Fail to find config file {}", url, e);
+      logger.warn("Fail to find config file {}", url, e);
       // update all data seriesPath
       conf.updatePath();
       return;
     }
 
-    LOGGER.info("Start to read config file {}", url);
+    logger.info("Start to read config file {}", url);
     Properties properties = new Properties();
     try {
       properties.load(inputStream);
       conf.setEnableStatMonitor(Boolean
           .parseBoolean(properties.getProperty("enable_stat_monitor",
-                  Boolean.toString(conf.isEnableStatMonitor()))));
+              Boolean.toString(conf.isEnableStatMonitor()))));
       conf.setBackLoopPeriodSec(Integer
           .parseInt(properties.getProperty("back_loop_period_in_second",
-                  Integer.toString(conf.getBackLoopPeriodSec()))));
+              Integer.toString(conf.getBackLoopPeriodSec()))));
       int statMonitorDetectFreqSec = Integer.parseInt(
           properties.getProperty("stat_monitor_detect_freq_in_second",
-                  Integer.toString(conf.getStatMonitorDetectFreqSec())));
+              Integer.toString(conf.getStatMonitorDetectFreqSec())));
       int statMonitorRetainIntervalSec = Integer.parseInt(
           properties.getProperty("stat_monitor_retain_interval_in_second",
-                  Integer.toString(conf.getStatMonitorRetainIntervalSec())));
+              Integer.toString(conf.getStatMonitorRetainIntervalSec())));
       // the conf value must > default value, or may cause system unstable
       if (conf.getStatMonitorDetectFreqSec() < statMonitorDetectFreqSec) {
         conf.setStatMonitorDetectFreqSec(statMonitorDetectFreqSec);
       } else {
-        LOGGER.info("The stat_monitor_detect_freq_sec value is smaller than default,"
+        logger.info("The stat_monitor_detect_freq_sec value is smaller than default,"
             + " use default value");
       }
 
       if (conf.getStatMonitorRetainIntervalSec() < statMonitorRetainIntervalSec) {
         conf.setStatMonitorRetainIntervalSec(statMonitorRetainIntervalSec);
       } else {
-        LOGGER.info("The stat_monitor_retain_interval_sec value is smaller than default,"
+        logger.info("The stat_monitor_retain_interval_sec value is smaller than default,"
             + " use default value");
       }
 
@@ -118,30 +126,35 @@ public class IoTDBDescriptor {
       conf.setEnableWal(Boolean.parseBoolean(properties.getProperty("enable_wal",
           Boolean.toString(conf.isEnableWal()))));
 
+      conf.setBaseDir(properties.getProperty("base_dir", conf.getBaseDir()));
+
+      conf.setSystemDir(FilePathUtils.regularizePath(conf.getBaseDir()) + "system");
+
+      conf.setDataDirs(properties.getProperty("data_dirs", conf.getDataDirs()[0])
+          .split(","));
+
+      conf.setWalFolder(properties.getProperty("wal_dir", conf.getWalFolder()));
+
+      conf.setMemtableNumber(Integer
+          .parseInt(properties.getProperty("memtable_number",
+              Integer.toString(conf.getMemtableNumber()))));
+
       conf.setFlushWalThreshold(Integer
           .parseInt(properties.getProperty("flush_wal_threshold",
-                  Integer.toString(conf.getFlushWalThreshold()))));
-      conf.setFlushWalPeriodInMs(Long
-          .parseLong(properties.getProperty("flush_wal_period_in_ms",
-                  Long.toString(conf.getFlushWalPeriodInMs()))));
+              Integer.toString(conf.getFlushWalThreshold()))));
+
       conf.setForceWalPeriodInMs(Long
           .parseLong(properties.getProperty("force_wal_period_in_ms",
-                  Long.toString(conf.getForceWalPeriodInMs()))));
-
-      conf.setDataDir(properties.getProperty("data_dir", conf.getDataDir()));
-      conf.setBufferWriteDirs(properties.getProperty("tsfile_dir", conf.DEFAULT_TSFILE_DIR)
-          .split(","));
-      conf.setSysDir(properties.getProperty("sys_dir", conf.getSysDir()));
-      conf.setWalDir(properties.getProperty("wal_dir", conf.getWalDir()));
+              Long.toString(conf.getForceWalPeriodInMs()))));
+      conf.setWalBufferSize(Integer.parseInt(properties.getProperty("wal_buffer_size",
+          Integer.toString(conf.getWalBufferSize()))));
 
-      conf.setMultDirStrategyClassName(properties.getProperty("mult_dir_strategy",
-          conf.getMultDirStrategyClassName()));
+      conf.setMultiDirStrategyClassName(properties.getProperty("mult_dir_strategy",
+          conf.getMultiDirStrategyClassName()));
 
-      conf.setMaxOpenFolder(Integer.parseInt(properties.getProperty("max_opened_folder",
-              Integer.toString(conf.getMaxOpenFolder()))));
       conf.setMergeConcurrentThreads(Integer
           .parseInt(properties.getProperty("merge_concurrent_threads",
-                  Integer.toString(conf.getMergeConcurrentThreads()))));
+              Integer.toString(conf.getMergeConcurrentThreads()))));
       if (conf.getMergeConcurrentThreads() <= 0
           || conf.getMergeConcurrentThreads() > Runtime.getRuntime().availableProcessors()) {
         conf.setMergeConcurrentThreads(Runtime.getRuntime().availableProcessors());
@@ -150,123 +163,60 @@ public class IoTDBDescriptor {
       conf.setFetchSize(Integer.parseInt(properties.getProperty("fetch_size",
           Integer.toString(conf.getFetchSize()))));
 
-      conf.setPeriodTimeForFlush(Long.parseLong(
-          properties.getProperty("period_time_for_flush_in_second",
-                  Long.toString(conf.getPeriodTimeForFlush())).trim()));
-      conf.setPeriodTimeForMerge(Long.parseLong(
-          properties.getProperty("period_time_for_merge_in_second",
-              Long.toString(conf.getPeriodTimeForMerge())).trim()));
-      conf.setEnableTimingCloseAndMerge(Boolean.parseBoolean(properties
-          .getProperty("enable_timing_close_and_merge",
-                  Boolean.toString(conf.isEnableTimingCloseAndMerge())).trim()));
-
-      conf.setMemThresholdWarning((long) (Runtime.getRuntime().maxMemory() * Double.parseDouble(
-          properties.getProperty("mem_threshold_warning",
-                  Long.toString(conf.getMemThresholdWarning())).trim())));
-      conf.setMemThresholdDangerous((long) (Runtime.getRuntime().maxMemory() * Double.parseDouble(
-          properties.getProperty("mem_threshold_dangerous",
-                  Long.toString(conf.getMemThresholdDangerous())).trim())));
-
-      conf.setMemMonitorInterval(Long
-          .parseLong(properties.getProperty("mem_monitor_interval_in_ms",
-                  Long.toString(conf.getMemMonitorInterval())).trim()));
-
-      conf.setMemControllerType(Integer
-          .parseInt(properties.getProperty("mem_controller_type",
-                  Integer.toString(conf.getMemControllerType())).trim()));
-      conf.setMemControllerType(conf.getMemControllerType() >= ControllerType.values().length ? 0
-          : conf.getMemControllerType());
-
-      conf.setBufferwriteMetaSizeThreshold(Long.parseLong(properties
-          .getProperty("bufferwrite_meta_size_threshold",
-                  Long.toString(conf.getBufferwriteMetaSizeThreshold())).trim()));
-      conf.setBufferwriteFileSizeThreshold(Long.parseLong(properties
-          .getProperty("bufferwrite_file_size_threshold",
-                  Long.toString(conf.getBufferwriteFileSizeThreshold())).trim()));
-
-      conf.setOverflowMetaSizeThreshold(Long.parseLong(
-          properties.getProperty("overflow_meta_size_threshold",
-                  Long.toString(conf.getOverflowMetaSizeThreshold())).trim()));
-      conf.setOverflowFileSizeThreshold(Long.parseLong(
-          properties.getProperty("overflow_file_size_threshold",
-              Long.toString(conf.getOverflowFileSizeThreshold())).trim()));
+      conf.setTsFileSizeThreshold(Long.parseLong(properties
+          .getProperty("tsfile_size_threshold",
+              Long.toString(conf.getTsFileSizeThreshold())).trim()));
 
       conf.setSyncEnable(Boolean
           .parseBoolean(properties.getProperty("is_sync_enable",
-                  Boolean.toString(conf.isSyncEnable()))));
+              Boolean.toString(conf.isSyncEnable()))));
       conf.setSyncServerPort(Integer
           .parseInt(properties.getProperty("sync_server_port",
-                  Integer.toString(conf.getSyncServerPort())).trim()));
+              Integer.toString(conf.getSyncServerPort())).trim()));
       conf.setUpdateHistoricalDataPossibility(Boolean.parseBoolean(
           properties.getProperty("update_historical_data_possibility",
-                  Boolean.toString(conf.isSyncEnable()))));
-      conf.setIpWhiteList(properties.getProperty("IP_white_list", conf.getIpWhiteList()));
-
-      if (conf.getMemThresholdWarning() <= 0) {
-        conf.setMemThresholdWarning(IoTDBConstant.MEM_THRESHOLD_WARNING_DEFAULT);
-      }
-      if (conf.getMemThresholdDangerous() < conf.getMemThresholdWarning()) {
-        conf.setMemThresholdDangerous(Math.max(conf.getMemThresholdWarning(),
-            IoTDBConstant.MEM_THRESHOLD_DANGEROUS_DEFAULT));
-      }
+              Boolean.toString(conf.isSyncEnable()))));
+      conf.setIpWhiteList(properties.getProperty("ip_white_list", conf.getIpWhiteList()));
 
       conf.setConcurrentFlushThread(Integer
           .parseInt(properties.getProperty("concurrent_flush_thread",
-                  Integer.toString(conf.getConcurrentFlushThread()))));
+              Integer.toString(conf.getConcurrentFlushThread()))));
       if (conf.getConcurrentFlushThread() <= 0) {
         conf.setConcurrentFlushThread(Runtime.getRuntime().availableProcessors());
       }
 
-      conf.setEnableMemMonitor(Boolean
-          .parseBoolean(properties.getProperty("enable_mem_monitor",
-                  Boolean.toString(conf.isEnableMemMonitor())).trim()));
-      conf.setEnableSmallFlush(Boolean
-          .parseBoolean(properties.getProperty("enable_small_flush",
-                  Boolean.toString(conf.isEnableSmallFlush())).trim()));
-      conf.setSmallFlushInterval(Long
-          .parseLong(properties.getProperty("small_flush_interval",
-                  Long.toString(conf.getSmallFlushInterval())).trim()));
-      conf.setExternalSortThreshold(Integer.parseInt(
-          properties.getProperty("external_sort_threshold",
-                  Integer.toString(conf.getExternalSortThreshold())).trim()));
       conf.setmManagerCacheSize(Integer
           .parseInt(properties.getProperty("schema_manager_cache_size",
-                  Integer.toString(conf.getmManagerCacheSize())).trim()));
-
-      int maxLogEntrySize = Integer
-          .parseInt(properties.getProperty("max_log_entry_size",
-                  Integer.toString(conf.getMaxLogEntrySize())).trim());
-      conf.setMaxLogEntrySize(maxLogEntrySize > 0 ? maxLogEntrySize :
-          conf.getMaxLogEntrySize());
+              Integer.toString(conf.getmManagerCacheSize())).trim()));
 
       conf.setLanguageVersion(properties.getProperty("language_version",
           conf.getLanguageVersion()).trim());
 
-      String tmpTimeZone = properties.getProperty("time_zone", conf.getZoneID().toString());
-      try {
-        conf.setZoneID(ZoneId.of(tmpTimeZone.trim()));
-        LOGGER.info("Time zone has been set to {}", conf.getZoneID());
-      } catch (Exception e) {
-        LOGGER.error("Time zone format error {}, use default configuration {}", tmpTimeZone,
-            conf.getZoneID(), e);
+      if (properties.containsKey("chunk_buffer_pool_enable")) {
+        conf.setChunkBufferPoolEnable(Boolean
+            .parseBoolean(properties.getProperty("chunk_buffer_pool_enable")));
       }
+      String tmpTimeZone = properties.getProperty("time_zone", conf.getZoneID().toString());
+      conf.setZoneID(ZoneId.of(tmpTimeZone.trim()));
+      logger.info("Time zone has been set to {}", conf.getZoneID());
 
     } catch (IOException e) {
-      LOGGER.warn("Cannot load config file because, use default configuration", e);
+      logger.warn("Cannot load config file because, use default configuration", e);
     } catch (Exception e) {
-      LOGGER.warn("Incorrect format in config file, use default configuration", e);
+      logger.warn("Incorrect format in config file, use default configuration", e);
     } finally {
       // update all data seriesPath
       conf.updatePath();
       try {
         inputStream.close();
       } catch (IOException e) {
-        LOGGER.error("Fail to close config file input stream because ", e);
+        logger.error("Fail to close config file input stream because ", e);
       }
     }
   }
 
   private static class IoTDBDescriptorHolder {
+
     private static final IoTDBDescriptor INSTANCE = new IoTDBDescriptor();
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java
deleted file mode 100644
index d792c37..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.conf.directories;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy;
-import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The main class of multiple directories. Used to allocate folders to data files.
- *
- * @author East
- */
-public class Directories {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(Directories.class);
-
-  private List<String> tsfileFolders;
-  private DirectoryStrategy strategy;
-
-  private Directories() {
-    tsfileFolders = new ArrayList<>(
-        Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getBufferWriteDirs()));
-    initFolders();
-
-    String strategyName = "";
-    try {
-      strategyName = IoTDBDescriptor.getInstance().getConfig().getMultDirStrategyClassName();
-      Class<?> clazz = Class.forName(strategyName);
-      strategy = (DirectoryStrategy) clazz.newInstance();
-      strategy.init(tsfileFolders);
-    } catch (Exception e) {
-      LOGGER.error("can't find strategy {} for mult-directories.", strategyName, e);
-    }
-  }
-
-  public static Directories getInstance() {
-    return DirectoriesHolder.INSTANCE;
-  }
-
-  private void initFolders() {
-    for (String folder : tsfileFolders) {
-      File file = new File(folder);
-      if (file.mkdirs()) {
-        LOGGER.info("folder {} in tsfileFolders doesn't exist, create it", file.getPath());
-      }
-    }
-  }
-
-  // only used by test
-  public String getFolderForTest() {
-    return tsfileFolders.get(0);
-  }
-
-  // only used by test
-  public void setFolderForTest(String path) {
-    tsfileFolders.set(0, path);
-  }
-
-  public String getNextFolderForTsfile() throws DiskSpaceInsufficientException {
-    return getTsFileFolder(getNextFolderIndexForTsFile());
-  }
-
-  /**
-   * get next folder index for TsFile.
-   *
-   * @return next folder index
-   */
-  public int getNextFolderIndexForTsFile() throws DiskSpaceInsufficientException {
-    return strategy.nextFolderIndex();
-  }
-
-  public String getTsFileFolder(int index) {
-    return tsfileFolders.get(index);
-  }
-
-  public int getTsFileFolderIndex(String folder) {
-    return tsfileFolders.indexOf(folder);
-  }
-
-  public List<String> getAllTsFileFolders() {
-    return tsfileFolders;
-  }
-
-  private static class DirectoriesHolder {
-    private static final Directories INSTANCE = new Directories();
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
new file mode 100644
index 0000000..9f4c1b2
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
@@ -0,0 +1,156 @@
+/**
+ * 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 org.apache.iotdb.db.conf.directories;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The main class of multiple directories. Used to allocate folders to data files.
+ */
+public class DirectoryManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(DirectoryManager.class);
+
+  private List<String> sequenceFileFolders;
+  private List<String> unsequenceFileFolders;
+  private DirectoryStrategy sequenceStrategy;
+  private DirectoryStrategy unsequenceStrategy;
+
+  private DirectoryManager() {
+    sequenceFileFolders =
+        new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
+    for (int i = 0; i < sequenceFileFolders.size(); i++) {
+      sequenceFileFolders.set(i, sequenceFileFolders.get(i) + File.separator + "sequence");
+    }
+    mkDirs(sequenceFileFolders);
+
+    unsequenceFileFolders =
+        new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
+    for (int i = 0; i < unsequenceFileFolders.size(); i++) {
+      unsequenceFileFolders.set(i, unsequenceFileFolders.get(i) + File.separator + "unsequence");
+    }
+    mkDirs(unsequenceFileFolders);
+
+    String strategyName = "";
+    try {
+      strategyName = IoTDBDescriptor.getInstance().getConfig().getMultiDirStrategyClassName();
+      Class<?> clazz = Class.forName(strategyName);
+      sequenceStrategy = (DirectoryStrategy) clazz.newInstance();
+      sequenceStrategy.init(sequenceFileFolders);
+      unsequenceStrategy = (DirectoryStrategy) clazz.newInstance();
+      unsequenceStrategy.init(unsequenceFileFolders);
+    } catch (Exception e) {
+      logger.error("can't find sequenceStrategy {} for mult-directories.", strategyName, e);
+    }
+  }
+
+  public static DirectoryManager getInstance() {
+    return DirectoriesHolder.INSTANCE;
+  }
+
+  private void mkDirs(List<String> folders) {
+    for (String folder : folders) {
+      File file = new File(folder);
+      if (file.mkdirs()) {
+        logger.info("folder {} doesn't exist, create it", file.getPath());
+      }
+    }
+  }
+
+  // only used by test
+  public String getSequenceFolderForTest() {
+    return sequenceFileFolders.get(0);
+  }
+
+  // only used by test
+  public void setSequenceFolderForTest(String path) {
+    sequenceFileFolders.set(0, path);
+  }
+
+  public String getNextFolderForSequenceFile() throws DiskSpaceInsufficientException {
+    return getSequenceFileFolder(getNextFolderIndexForSequenceFile());
+  }
+
+  /**
+   * get next folder index for TsFile.
+   *
+   * @return next folder index
+   */
+  public int getNextFolderIndexForSequenceFile() throws DiskSpaceInsufficientException {
+    return sequenceStrategy.nextFolderIndex();
+  }
+
+  public String getSequenceFileFolder(int index) {
+    return sequenceFileFolders.get(index);
+  }
+
+  public int getSequenceFileFolderIndex(String folder) {
+    return sequenceFileFolders.indexOf(folder);
+  }
+
+  public List<String> getAllSequenceFileFolders() {
+    return sequenceFileFolders;
+  }
+
+  private static class DirectoriesHolder {
+    private static final DirectoryManager INSTANCE = new DirectoryManager();
+  }
+
+  public String getWALFolder() {
+    return IoTDBDescriptor.getInstance().getConfig().getWalFolder();
+  }
+
+  public String getNextFolderForUnSequenceFile() throws DiskSpaceInsufficientException {
+    return getUnSequenceFileFolder(getNextFolderIndexForUnSequenceFile());
+  }
+
+  /**
+   * get next folder index for OverflowFile.
+   *
+   * @return next folder index
+   */
+  public int getNextFolderIndexForUnSequenceFile() throws DiskSpaceInsufficientException {
+    return unsequenceStrategy.nextFolderIndex();
+  }
+
+  public String getUnSequenceFileFolder(int index) {
+    return unsequenceFileFolders.get(index);
+  }
+
+  public int getUnSequenceFileFolderIndex(String folder) {
+    return unsequenceFileFolders.indexOf(folder);
+  }
+
+  public List<String> getAllUnSequenceFileFolders() {
+    return unsequenceFileFolders;
+  }
+
+  // only used by test
+  public String getUnSequenceFolderForTest() {
+    return unsequenceFileFolders.get(0);
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
index dd397b8..1da0478 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
@@ -27,12 +27,10 @@ import org.slf4j.LoggerFactory;
 /**
  * The basic class of all the strategies of multiple directories. If a user wants to define his own
  * strategy, his strategy has to extend this class and implement the abstract method.
- *
- * @author East
  */
 public abstract class DirectoryStrategy {
 
-  protected static final Logger LOGGER = LoggerFactory.getLogger(DirectoryStrategy.class);
+  protected static final Logger logger = LoggerFactory.getLogger(DirectoryStrategy.class);
 
   /**
    * All the folders of data files, should be init once the subclass is created.
@@ -76,8 +74,12 @@ public abstract class DirectoryStrategy {
   }
 
   protected long getUsableSpace(String dir) {
-    long space = new File(dir).getFreeSpace();
-    LOGGER.trace("Folder {} has {} available bytes.", dir, space);
+    File file = new File(dir);
+    if (!file.exists() && !file.mkdirs()) {
+      return 0;
+    }
+    long space = file.getFreeSpace();
+    logger.trace("Folder {} has {} available bytes.", dir, space);
     return space;
   }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/MinFolderOccupiedSpaceFirstStrategy.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/MinFolderOccupiedSpaceFirstStrategy.java
index 81f0f73..eda2565 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/MinFolderOccupiedSpaceFirstStrategy.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/strategy/MinFolderOccupiedSpaceFirstStrategy.java
@@ -69,7 +69,7 @@ public class MinFolderOccupiedSpaceFirstStrategy extends DirectoryStrategy {
             .sum();
       }
     } catch (IOException e) {
-      LOGGER.error("Cannot calculate occupied space for seriesPath {}.", path, e);
+      logger.error("Cannot calculate occupied space for seriesPath {}.", path, e);
     }
 
     return size / DATA_SIZE_SHIFT;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/Processor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/Processor.java
deleted file mode 100644
index 5c515ff..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/Processor.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine;
-
-import java.io.IOException;
-import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor;
-import org.apache.iotdb.db.engine.filenode.FileNodeProcessor;
-import org.apache.iotdb.db.exception.ProcessorException;
-
-/**
- * Processor is used for implementing different processor with different operation.<br>
- *
- * @see BufferWriteProcessor
- * @see FileNodeProcessor
- */
-public abstract class Processor {
-
-  private final ReadWriteLock lock;
-  protected String processorName;
-
-  /**
-   * Construct processor using name space seriesPath
-   *
-   * @param processorName
-   */
-  public Processor(String processorName) {
-    this.processorName = processorName;
-    this.lock = new ReentrantReadWriteLock();
-  }
-
-  /**
-   * Release the read lock
-   */
-  public void readUnlock() {
-    lock.readLock().unlock();
-  }
-
-  /**
-   * Acquire the read lock
-   */
-  public void readLock() {
-    lock.readLock().lock();
-  }
-
-  /**
-   * Acquire the write lock
-   */
-  public void writeLock() {
-    lock.writeLock().lock();
-  }
-
-  /**
-   * Release the write lock
-   */
-  public void writeUnlock() {
-    lock.writeLock().unlock();
-  }
-
-  /**
-   * @param isWriteLock
-   *            true acquire write lock, false acquire read lock
-   */
-  public void lock(boolean isWriteLock) {
-    if (isWriteLock) {
-      lock.writeLock().lock();
-    } else {
-      lock.readLock().lock();
-    }
-  }
-
-  public boolean tryLock(boolean isWriteLock) {
-    if (isWriteLock) {
-      return tryWriteLock();
-    } else {
-      return tryReadLock();
-    }
-  }
-
-  /**
-   * @param isWriteUnlock
-   *            true release write lock, false release read unlock
-   */
-  public void unlock(boolean isWriteUnlock) {
-    if (isWriteUnlock) {
-      writeUnlock();
-    } else {
-      readUnlock();
-    }
-  }
-
-  /**
-   * Get the name space seriesPath
-   *
-   * @return
-   */
-  public String getProcessorName() {
-    return processorName;
-  }
-
-  /**
-   * Try to get the write lock
-   *
-   * @return
-   */
-  public boolean tryWriteLock() {
-    return lock.writeLock().tryLock();
-  }
-
-  /**
-   * Try to get the read lock
-   *
-   * @return
-   */
-  public boolean tryReadLock() {
-    return lock.readLock().tryLock();
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((processorName == null) ? 0 : processorName.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    Processor other = (Processor) obj;
-    if (processorName == null) {
-      if (other.processorName != null) {
-        return false;
-      }
-    } else if (!processorName.equals(other.processorName)) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Judge whether this processor can be closed.
-   *
-   * @return true if subclass doesn't have other implementation.
-   */
-  public abstract boolean canBeClosed();
-
-  /**
-   * call flush operation asynchronously
-   * @return a future that returns true if successfully, otherwise false.
-   * @throws IOException
-   */
-  public abstract Future<Boolean> flush() throws IOException;
-
-  /**
-   * Close the processor.<br>
-   * Notice: Thread is not safe
-   *
-   * @throws IOException
-   * @throws ProcessorException
-   */
-  public abstract void close() throws ProcessorException;
-
-  public abstract long memoryUsage();
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
new file mode 100644
index 0000000..d27b0a3
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -0,0 +1,356 @@
+/**
+ * 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 org.apache.iotdb.db.engine;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.MetadataErrorException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.exception.StorageEngineFailureException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.service.IService;
+import org.apache.iotdb.db.service.ServiceType;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageEngine implements IService {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageEngine.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private volatile boolean readOnly = false;
+
+  /**
+   * a folder (system/storage_groups/ by default) that persist system info. Each Storage Processor will have a
+   * subfolder under the systemDir.
+   */
+  private final String systemDir;
+
+  /**
+   * storage group name -> storage group processor
+   */
+  private final ConcurrentHashMap<String, StorageGroupProcessor> processorMap = new ConcurrentHashMap<>();
+
+  private static final StorageEngine INSTANCE = new StorageEngine();
+
+  public static StorageEngine getInstance() {
+    return INSTANCE;
+  }
+
+  private StorageEngine() {
+    systemDir = FilePathUtils.regularizePath(config.getSystemDir()) + "storage_groups";
+    // create systemDir
+    try {
+      FileUtils.forceMkdir(new File(systemDir));
+    } catch (IOException e) {
+      throw new StorageEngineFailureException("create system directory failed!");
+    }
+
+    /**
+     * recover all storage group processors.
+     */
+    try {
+      List<String> storageGroups = MManager.getInstance().getAllStorageGroupNames();
+      for (String storageGroup: storageGroups) {
+        StorageGroupProcessor processor = new StorageGroupProcessor(systemDir, storageGroup);
+        logger.info("Storage Group Processor {} is recovered successfully", storageGroup);
+        processorMap.put(storageGroup, processor);
+      }
+    } catch (ProcessorException | MetadataErrorException e) {
+      logger.error("init a storage group processor failed. ", e);
+      throw new StorageEngineFailureException(e);
+    }
+  }
+
+  @Override
+  public void start() {
+    // nothing to be done
+  }
+
+  @Override
+  public void stop() {
+    syncCloseAllProcessor();
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.STORAGE_ENGINE_SERVICE;
+  }
+
+
+  private StorageGroupProcessor getProcessor(String path) throws StorageEngineException {
+    String storageGroupName = "";
+    try {
+      storageGroupName = MManager.getInstance().getStorageGroupNameByPath(path);
+      StorageGroupProcessor processor;
+      processor = processorMap.get(storageGroupName);
+      if (processor == null) {
+        storageGroupName = storageGroupName.intern();
+        synchronized (storageGroupName) {
+          processor = processorMap.get(storageGroupName);
+          if (processor == null) {
+            logger.debug("construct a processor instance, the storage group is {}, Thread is {}",
+                storageGroupName, Thread.currentThread().getId());
+            processor = new StorageGroupProcessor(systemDir, storageGroupName);
+            synchronized (processorMap) {
+              processorMap.put(storageGroupName, processor);
+            }
+          }
+        }
+      }
+      return processor;
+    } catch (PathErrorException | ProcessorException e) {
+      logger.error("Fail to get StorageGroupProcessor {}", storageGroupName,  e);
+      throw new StorageEngineException(e);
+    }
+  }
+
+
+  /**
+   * This function is just for unit test.
+   */
+  public synchronized void reset() throws IOException {
+    processorMap.clear();
+    readOnly = false;
+  }
+
+
+  /**
+   * execute an InsertPlan on a storage group.
+   *
+   * @param insertPlan physical plan of insertion
+   * @return true if and only if this insertion succeeds
+   */
+  public boolean insert(InsertPlan insertPlan) throws StorageEngineException {
+
+    if (readOnly) {
+      throw new StorageEngineException(
+          "Current system mode is read only, does not support insertion");
+    }
+
+    StorageGroupProcessor storageGroupProcessor;
+    try {
+      storageGroupProcessor = getProcessor(insertPlan.getDeviceId());
+    } catch (Exception e) {
+      logger.warn("get StorageGroupProcessor of device {} failed, because {}",
+          insertPlan.getDeviceId(),
+          e.getMessage(), e);
+      throw new StorageEngineException(e);
+    }
+
+    // TODO monitor: update statistics
+    return storageGroupProcessor.insert(insertPlan);
+  }
+
+  /**
+   * only for unit test
+   */
+  public void asyncFlushAndSealAllFiles() {
+    synchronized (processorMap) {
+      for (StorageGroupProcessor storageGroupProcessor : processorMap.values()) {
+        storageGroupProcessor.putAllWorkingTsFileProcessorIntoClosingList();
+      }
+    }
+  }
+
+
+  /**
+   * flush command
+   * Sync asyncCloseOneProcessor all file node processors.
+   */
+  public void syncCloseAllProcessor() {
+    logger.info("Start closing all storage group processor");
+    synchronized (processorMap){
+      for(StorageGroupProcessor processor: processorMap.values()){
+        processor.waitForAllCurrentTsFileProcessorsClosed();
+      }
+    }
+  }
+
+  /**
+   * update data.
+   */
+  public void update(String deviceId, String measurementId, long startTime, long endTime,
+      TSDataType type, String v) {
+    // TODO
+  }
+
+  /**
+   * delete data of timeseries "{deviceId}.{measurementId}" with time <= timestamp.
+   */
+  public void delete(String deviceId, String measurementId, long timestamp)
+      throws StorageEngineException {
+
+    if (readOnly) {
+      throw new StorageEngineException(
+          "Current system mode is read only, does not support deletion");
+    }
+
+    StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
+    try {
+      storageGroupProcessor.delete(deviceId, measurementId, timestamp);
+    } catch (IOException e) {
+      throw new StorageEngineException(e);
+    }
+  }
+
+
+  /**
+   * begin a query on a given deviceId. Any TsFile contains such device should not be deleted at
+   * once after merge.
+   *
+   * @param deviceId queried deviceId
+   * @return a token for the query.
+   */
+  public int beginQuery(String deviceId) throws StorageEngineException {
+    // TODO implement it when developing the merge function
+    return -1;
+  }
+
+  /**
+   * end query on a given deviceId. If some TsFile has been merged and this query is the
+   * last query using it, the TsFile can be deleted safely.
+   */
+  public void endQuery(String deviceId, int token) throws StorageEngineException {
+    // TODO  implement it when developing the merge function
+  }
+
+  /**
+   * query data.
+   */
+  public QueryDataSource query(SingleSeriesExpression seriesExpression, QueryContext context)
+      throws StorageEngineException {
+    //TODO use context.
+    String deviceId = seriesExpression.getSeriesPath().getDevice();
+    String measurementId = seriesExpression.getSeriesPath().getMeasurement();
+    StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
+    return storageGroupProcessor.query(deviceId, measurementId, context);
+  }
+
+  /**
+   * Append one specified tsfile to the storage group. <b>This method is only provided for
+   * transmission module</b>
+   *
+   * @param storageGroupName the seriesPath of storage group
+   * @param appendFile the appended tsfile information
+   */
+  @SuppressWarnings("unused") // reimplement sync module
+  public boolean appendFileToStorageGroupProcessor(String storageGroupName, TsFileResource appendFile,
+      String appendFilePath) throws StorageEngineException {
+    // TODO reimplement sync module
+    return true;
+  }
+
+  /**
+   * get all overlap tsfiles which are conflict with the appendFile.
+   *
+   * @param storageGroupName the seriesPath of storage group
+   * @param appendFile the appended tsfile information
+   */
+  @SuppressWarnings("unused") // reimplement sync module
+  public List<String> getOverlapFiles(String storageGroupName, TsFileResource appendFile,
+      String uuid) throws StorageEngineException {
+    // TODO reimplement sync module
+    return Collections.emptyList();
+  }
+
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
+  public void setReadOnly(boolean readOnly) {
+    this.readOnly = readOnly;
+  }
+
+  /**
+   * merge all storage groups.
+   *
+   * @throws StorageEngineException StorageEngineException
+   */
+  public void mergeAll() throws StorageEngineException {
+    if (readOnly) {
+      throw new StorageEngineException("Current system mode is read only, does not support merge");
+    }
+    // TODO
+  }
+
+  /**
+   * delete all data files (both memory data and file on disk) in a storage group.
+   * It is used when there is no timeseries (which are all deleted) in this storage group)
+   */
+  public void deleteAllDataFilesInOneStorageGroup(String storageGroupName) {
+    if (processorMap.containsKey(storageGroupName)) {
+      syncDeleteDataFiles(storageGroupName);
+    }
+  }
+
+  private void syncDeleteDataFiles(String storageGroupName) {
+    logger.info("Force to delete the data in storage group processor {}", storageGroupName);
+    StorageGroupProcessor processor = processorMap.get(storageGroupName);
+    processor.syncDeleteDataFiles();
+  }
+
+  /**
+   * add time series.
+   */
+  public void addTimeSeries(Path path, TSDataType dataType, TSEncoding encoding,
+      CompressionType compressor, Map<String, String> props) throws StorageEngineException {
+    StorageGroupProcessor storageGroupProcessor = getProcessor(path.getDevice());
+    storageGroupProcessor
+        .addMeasurement(path.getMeasurement(), dataType, encoding, compressor, props);
+  }
+
+
+  /**
+   * delete all data of storage groups' timeseries.
+   */
+  public synchronized boolean deleteAll() {
+    logger.info("Start deleting all storage groups' timeseries");
+    try {
+      for (String storageGroup : MManager.getInstance().getAllStorageGroupNames()) {
+        this.deleteAllDataFilesInOneStorageGroup(storageGroup);
+      }
+    } catch (MetadataErrorException e) {
+      logger.error("delete storage groups failed.", e);
+      return false;
+    }
+    return true;
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/Action.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/Action.java
deleted file mode 100644
index af65a8a..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/Action.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.bufferwrite;
-
-/**
- * Action interface.
- */
-
-public interface Action {
-
-  void act() throws ActionException;
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/ActionException.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/ActionException.java
deleted file mode 100644
index 1d76d9f..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/ActionException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.bufferwrite;
-
-public class ActionException extends Exception{
-
-  public ActionException(String message) {
-    super(message);
-  }
-
-  public ActionException(Throwable cause) {
-    super(cause);
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java
deleted file mode 100644
index a8e1e58..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java
+++ /dev/null
@@ -1,561 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.bufferwrite;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.Processor;
-import org.apache.iotdb.db.engine.filenode.FileNodeManager;
-import org.apache.iotdb.db.engine.memcontrol.BasicMemController;
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
-import org.apache.iotdb.db.engine.memtable.MemTableFlushUtil;
-import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
-import org.apache.iotdb.db.engine.pool.FlushManager;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.engine.version.VersionController;
-import org.apache.iotdb.db.exception.BufferWriteProcessorException;
-import org.apache.iotdb.db.qp.constant.DatetimeUtils;
-import org.apache.iotdb.db.utils.ImmediateFuture;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
-import org.apache.iotdb.db.writelog.node.WriteLogNode;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.FileSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BufferWriteProcessor extends Processor {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(BufferWriteProcessor.class);
-  private RestorableTsFileIOWriter writer;
-  private FileSchema fileSchema;
-  private volatile Future<Boolean> flushFuture = new ImmediateFuture<>(true);
-  private ReentrantLock flushQueryLock = new ReentrantLock();
-  private AtomicLong memSize = new AtomicLong();
-  private long memThreshold = TSFileDescriptor.getInstance().getConfig().groupSizeInByte;
-  private IMemTable workMemTable;
-  private IMemTable flushMemTable;
-  private Action bufferwriteFlushAction;
-  private Action bufferwriteCloseAction;
-  private Action filenodeFlushAction;
-
-  //lastFlushTime time unit: nanosecond
-  private long lastFlushTime = -1;
-  private long valueCount = 0;
-
-  private String baseDir;
-  private String insertFilePath;
-  private String bufferWriteRelativePath;
-
-  private WriteLogNode logNode;
-  private VersionController versionController;
-
-  private boolean isClosed = true;
-  private boolean isFlush = false;
-
-  /**
-   * constructor of BufferWriteProcessor.
-   *
-   * @param baseDir base dir
-   * @param processorName processor name
-   * @param fileName file name
-   * @param parameters parameters in Map(String, Action) structure
-   * @param fileSchema file schema
-   * @throws BufferWriteProcessorException BufferWriteProcessorException
-   */
-  public BufferWriteProcessor(String baseDir, String processorName, String fileName,
-      Map<String, Action> parameters, VersionController versionController,
-      FileSchema fileSchema) throws BufferWriteProcessorException {
-    super(processorName);
-    this.fileSchema = fileSchema;
-    this.baseDir = baseDir;
-
-    bufferwriteFlushAction = parameters.get(FileNodeConstants.BUFFERWRITE_FLUSH_ACTION);
-    bufferwriteCloseAction = parameters.get(FileNodeConstants.BUFFERWRITE_CLOSE_ACTION);
-    filenodeFlushAction = parameters.get(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION);
-
-    reopen(fileName);
-    try {
-      getLogNode();
-    } catch (IOException e) {
-      throw new BufferWriteProcessorException(e);
-    }
-    this.versionController = versionController;
-
-  }
-
-  public void reopen(String fileName) throws BufferWriteProcessorException {
-    if (!isClosed) {
-      return;
-    }
-    new File(baseDir, processorName).mkdirs();
-    this.insertFilePath = Paths.get(baseDir, processorName, fileName).toString();
-    bufferWriteRelativePath = processorName + File.separatorChar + fileName;
-    try {
-      writer = new RestorableTsFileIOWriter(processorName, insertFilePath);
-    } catch (IOException e) {
-      throw new BufferWriteProcessorException(e);
-    }
-    if (workMemTable == null) {
-      workMemTable = new PrimitiveMemTable();
-    } else {
-      workMemTable.clear();
-    }
-    isClosed = false;
-    isFlush = false;
-  }
-
-  public void checkOpen() throws BufferWriteProcessorException {
-    if (isClosed) {
-      throw new BufferWriteProcessorException("BufferWriteProcessor already closed");
-    }
-  }
-
-
-  /**
-   * write one data point to the buffer write.
-   *
-   * @param deviceId device name
-   * @param measurementId sensor name
-   * @param timestamp timestamp of the data point
-   * @param dataType the data type of the value
-   * @param value data point value
-   * @return true -the size of tsfile or metadata reaches to the threshold. false -otherwise
-   * @throws BufferWriteProcessorException if a flushing operation occurs and failed.
-   */
-  public boolean write(String deviceId, String measurementId, long timestamp, TSDataType dataType,
-      String value)
-      throws BufferWriteProcessorException {
-    checkOpen();
-    TSRecord record = new TSRecord(timestamp, deviceId);
-    DataPoint dataPoint = DataPoint.getDataPoint(dataType, measurementId, value);
-    record.addTuple(dataPoint);
-    return write(record);
-  }
-
-  /**
-   * wrete a ts record into the memtable. If the memory usage is beyond the memThreshold, an async
-   * flushing operation will be called.
-   *
-   * @param tsRecord data to be written
-   * @return FIXME what is the mean about the return value??
-   * @throws BufferWriteProcessorException if a flushing operation occurs and failed.
-   */
-  public boolean write(TSRecord tsRecord) throws BufferWriteProcessorException {
-    checkOpen();
-    long memUsage = MemUtils.getRecordSize(tsRecord);
-    BasicMemController.UsageLevel level = BasicMemController.getInstance()
-        .acquireUsage(this, memUsage);
-
-    String memory;
-    switch (level) {
-      case SAFE:
-        for (DataPoint dataPoint : tsRecord.dataPointList) {
-          workMemTable.write(tsRecord.deviceId, dataPoint.getMeasurementId(), dataPoint.getType(),
-              tsRecord.time,
-              dataPoint.getValue().toString());
-        }
-        valueCount++;
-        checkMemThreshold4Flush(memUsage);
-        return true;
-      case WARNING:
-        memory = MemUtils.bytesCntToStr(BasicMemController.getInstance().getTotalUsage());
-        LOGGER.warn("Memory usage will exceed warning threshold, current : {}.", memory);
-        for (DataPoint dataPoint : tsRecord.dataPointList) {
-          workMemTable.write(tsRecord.deviceId, dataPoint.getMeasurementId(), dataPoint.getType(),
-              tsRecord.time,
-              dataPoint.getValue().toString());
-        }
-        valueCount++;
-        try {
-          flush();
-        } catch (IOException e) {
-          throw new BufferWriteProcessorException(e);
-        }
-        return true;
-      case DANGEROUS:
-      default:
-        memory = MemUtils.bytesCntToStr(BasicMemController.getInstance().getTotalUsage());
-        LOGGER.warn("Memory usage will exceed dangerous threshold, current : {}.", memory);
-        return false;
-    }
-  }
-
-  private void checkMemThreshold4Flush(long addedMemory) throws BufferWriteProcessorException {
-    long newMem = memSize.addAndGet(addedMemory);
-    if (newMem > memThreshold) {
-      String usageMem = MemUtils.bytesCntToStr(newMem);
-      String threshold = MemUtils.bytesCntToStr(memThreshold);
-      String processorName = getProcessorName();
-      LOGGER.info("The usage of memory {} in bufferwrite processor {} reaches the threshold {}",
-          usageMem, processorName, threshold);
-      try {
-        flush();
-      } catch (IOException e) {
-        LOGGER.error("Flush bufferwrite error.", e);
-        throw new BufferWriteProcessorException(e);
-      }
-    }
-  }
-
-  /**
-   * get the one (or two) chunk(s) in the memtable ( and the other one in flushing status and then
-   * compact them into one TimeValuePairSorter). Then get its (or their) ChunkMetadata(s).
-   *
-   * @param deviceId device id
-   * @param measurementId sensor id
-   * @param dataType data type
-   * @return corresponding chunk data and chunk metadata in memory
-   */
-  public Pair<ReadOnlyMemChunk, List<ChunkMetaData>> queryBufferWriteData(String deviceId,
-      String measurementId, TSDataType dataType, Map<String, String> props)
-      throws BufferWriteProcessorException {
-    checkOpen();
-    flushQueryLock.lock();
-    try {
-      MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
-      if (flushMemTable != null) {
-        memSeriesLazyMerger.addMemSeries(flushMemTable.query(deviceId, measurementId, dataType, props));
-      }
-      memSeriesLazyMerger.addMemSeries(workMemTable.query(deviceId, measurementId, dataType, props));
-      // memSeriesLazyMerger has handled the props,
-      // so we do not need to handle it again in the following readOnlyMemChunk
-      ReadOnlyMemChunk timeValuePairSorter = new ReadOnlyMemChunk(dataType, memSeriesLazyMerger,
-          Collections.emptyMap());
-      return new Pair<>(timeValuePairSorter,
-          writer.getMetadatas(deviceId, measurementId, dataType));
-    } finally {
-      flushQueryLock.unlock();
-    }
-  }
-
-  private void switchWorkToFlush() {
-    flushQueryLock.lock();
-    try {
-      IMemTable temp = flushMemTable == null ? new PrimitiveMemTable() : flushMemTable;
-      flushMemTable = workMemTable;
-      workMemTable = temp;
-      isFlush = true;
-    } finally {
-      flushQueryLock.unlock();
-    }
-  }
-
-  private void switchFlushToWork() {
-    flushQueryLock.lock();
-    try {
-      flushMemTable.clear();
-      writer.appendMetadata();
-      isFlush = false;
-    } finally {
-      flushQueryLock.unlock();
-    }
-  }
-
-
-  /**
-   * the caller mast guarantee no other concurrent caller entering this function.
-   *
-   * @param displayMessage message that will appear in system log.
-   * @param version the operation version that will tagged on the to be flushed memtable
-   * (i.e., ChunkGroup)
-   * @return true if successfully.
-   */
-  private boolean flushTask(String displayMessage, long version) {
-    boolean result;
-    long flushStartTime = System.currentTimeMillis();
-    LOGGER.info("The bufferwrite processor {} starts flushing {}.", getProcessorName(),
-        displayMessage);
-    try {
-      if (flushMemTable != null && !flushMemTable.isEmpty()) {
-        // flush data
-        MemTableFlushUtil.flushMemTable(fileSchema, writer, flushMemTable,
-            version);
-        // write restore information
-        writer.flush();
-      }
-
-      filenodeFlushAction.act();
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        logNode.notifyEndFlush(null);
-      }
-      result = true;
-    } catch (Exception e) {
-      LOGGER.error(
-          "The bufferwrite processor {} failed to flush {}, when calling the filenodeFlushAction.",
-          getProcessorName(), displayMessage, e);
-      result = false;
-    } finally {
-      switchFlushToWork();
-      LOGGER.info("The bufferwrite processor {} ends flushing {}.", getProcessorName(),
-            displayMessage);
-    }
-    if (LOGGER.isInfoEnabled()) {
-      long flushEndTime = System.currentTimeMillis();
-      LOGGER.info(
-          "The bufferwrite processor {} flush {}, start time is {}, flush end time is {}, "
-              + "flush time consumption is {}ms",
-          getProcessorName(), displayMessage,
-          DatetimeUtils.convertMillsecondToZonedDateTime(flushStartTime),
-          DatetimeUtils.convertMillsecondToZonedDateTime(flushEndTime),
-          flushEndTime - flushStartTime);
-    }
-    return result;
-  }
-
-  // keyword synchronized is added in this method, so that only one flush task can be submitted now.
-  @Override
-  public synchronized Future<Boolean> flush() throws IOException {
-    if (isClosed) {
-      throw new IOException("BufferWriteProcessor closed");
-    }
-    // statistic information for flush
-    if (lastFlushTime > 0) {
-      if (LOGGER.isInfoEnabled()) {
-        long thisFlushTime = System.currentTimeMillis();
-        LOGGER.info(
-            "The bufferwrite processor {}: last flush time is {}, this flush time is {}, "
-                + "flush time interval is {}s", getProcessorName(),
-            DatetimeUtils.convertMillsecondToZonedDateTime(lastFlushTime / 1000),
-            DatetimeUtils.convertMillsecondToZonedDateTime(thisFlushTime),
-            (thisFlushTime - lastFlushTime / 1000) / 1000);
-      }
-    }
-    lastFlushTime = System.nanoTime();
-    // check value count
-    // waiting for the end of last flush operation.
-    try {
-      flushFuture.get();
-    } catch (InterruptedException | ExecutionException e) {
-      throw new IOException(e);
-    }
-    if (valueCount > 0) {
-      // update the lastUpdatetime, prepare for flush
-      try {
-        bufferwriteFlushAction.act();
-      } catch (Exception e) {
-        LOGGER.error("Failed to flush bufferwrite row group when calling the action function.");
-        throw new IOException(e);
-      }
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        logNode.notifyStartFlush();
-      }
-      valueCount = 0;
-      switchWorkToFlush();
-      long version = versionController.nextVersion();
-      BasicMemController.getInstance().releaseUsage(this, memSize.get());
-      memSize.set(0);
-      // switch
-      flushFuture = FlushManager.getInstance().submit(() -> flushTask("asynchronously",
-          version));
-    } else {
-      flushFuture = new ImmediateFuture<>(true);
-    }
-    return flushFuture;
-  }
-
-  @Override
-  public boolean canBeClosed() {
-    return true;
-  }
-
-  @Override
-  public void close() throws BufferWriteProcessorException {
-    if (isClosed) {
-      return;
-    }
-    try {
-      long closeStartTime = System.currentTimeMillis();
-      // flush data and wait for finishing flush
-      flush().get();
-      // end file
-      writer.endFile(fileSchema);
-      writer = null;
-      workMemTable.clear();
-
-      // update the IntervalFile for interval list
-      bufferwriteCloseAction.act();
-      // flush the changed information for filenode
-      filenodeFlushAction.act();
-      // delete the restore for this bufferwrite processor
-      if (LOGGER.isInfoEnabled()) {
-        long closeEndTime = System.currentTimeMillis();
-        LOGGER.info(
-            "Close bufferwrite processor {}, the file name is {}, start time is {}, end time is {}, "
-                + "time consumption is {}ms",
-            getProcessorName(), insertFilePath,
-            DatetimeUtils.convertMillsecondToZonedDateTime(closeStartTime),
-            DatetimeUtils.convertMillsecondToZonedDateTime(closeEndTime),
-            closeEndTime - closeStartTime);
-      }
-      isClosed = true;
-    } catch (IOException e) {
-      LOGGER.error("Close the bufferwrite processor error, the bufferwrite is {}.",
-          getProcessorName(), e);
-      throw new BufferWriteProcessorException(e);
-    } catch (Exception e) {
-      LOGGER
-          .error("Failed to close the bufferwrite processor when calling the action function.", e);
-      throw new BufferWriteProcessorException(e);
-    }
-  }
-
-  @Override
-  public long memoryUsage() {
-    return memSize.get();
-  }
-
-  /**
-   * check if is flushing.
-   *
-   * @return True if flushing
-   */
-  public boolean isFlush() {
-    return isFlush;
-  }
-
-  /**
-   * get metadata size.
-   *
-   * @return The sum of all timeseries's metadata size within this file.
-   */
-  public long getMetaSize() {
-    // TODO : [MemControl] implement this
-    return 0;
-  }
-
-  /**
-   * get file size.
-   *
-   * @return The file size of the TsFile corresponding to this processor.
-   */
-  public long getFileSize() {
-    // TODO : save this variable to avoid object creation?
-    File file = new File(insertFilePath);
-    return file.length() + memoryUsage();
-  }
-
-  public String getBaseDir() {
-    return baseDir;
-  }
-
-
-  public String getFileRelativePath() {
-    return bufferWriteRelativePath;
-  }
-
-  private String getBufferwriteRestoreFilePath() {
-    return writer.getRestoreFilePath();
-  }
-
-  public boolean isNewProcessor() {
-    return writer.isNewResource();
-  }
-
-  public void setNewProcessor(boolean isNewProcessor) {
-    writer.setNewResource(isNewProcessor);
-  }
-
-  public WriteLogNode getLogNode() throws IOException {
-    if (logNode == null) {
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        logNode = MultiFileLogNodeManager.getInstance().getNode(
-            processorName + IoTDBConstant.BUFFERWRITE_LOG_NODE_SUFFIX,
-            getBufferwriteRestoreFilePath(),
-            FileNodeManager.getInstance().getRestoreFilePath(processorName));
-      }
-    }
-    return logNode;
-  }
-
-  /**
-   * used for test. We can know when the flush() is called.
-   * @return the last flush() time. Time unit: nanosecond.
-   */
-  public long getLastFlushTime() {
-    return lastFlushTime;
-  }
-
-  /**
-   * used for test. We can block to wait for finishing flushing.
-   * @return the future of the flush() task.
-   */
-  public Future<Boolean> getFlushFuture() {
-    return flushFuture;
-  }
-
-  /**
-   * Delete data whose timestamp <= 'timestamp' and belonging to timeseries deviceId.measurementId.
-   * Delete data in both working MemTable and flushing MemTable.
-   *
-   * @param deviceId the deviceId of the timeseries to be deleted.
-   * @param measurementId the measurementId of the timeseries to be deleted.
-   * @param timestamp the upper-bound of deletion time.
-   */
-  public void delete(String deviceId, String measurementId, long timestamp)
-      throws BufferWriteProcessorException {
-    checkOpen();
-    workMemTable.delete(deviceId, measurementId, timestamp);
-    if (isFlush()) {
-      // flushing MemTable cannot be directly modified since another thread is reading it
-      flushMemTable = flushMemTable.copy();
-      flushMemTable.delete(deviceId, measurementId, timestamp);
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return this == o;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(super.hashCode());
-  }
-
-  @Override
-  public String toString() {
-    return "BufferWriteProcessor in " + insertFilePath;
-  }
-
-  public String getInsertFilePath() {
-    return insertFilePath;
-  }
-
-  public boolean isClosed() {
-    return isClosed;
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/FileNodeConstants.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/FileNodeConstants.java
deleted file mode 100644
index 483f7f1..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/FileNodeConstants.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.bufferwrite;
-
-/**
- * Constants for using in bufferwrite, overflow and filenode.
- *
- */
-public class FileNodeConstants {
-
-  private FileNodeConstants(){
-  }
-
-  public static final String FILE_NODE_OPERATOR_TYPE = "OPERATOR_TYPE";
-  public static final String TIMESTAMP_KEY = "TIMESTAMP";
-  public static final String FILE_NODE = "FILE_NODE";
-  public static final String CLOSE_ACTION = "CLOSE_ACTION";
-
-  public static final String OVERFLOW_FLUSH_ACTION = "OVERFLOW_FLUSH_ACTION";
-  public static final String BUFFERWRITE_FLUSH_ACTION = "BUFFERWRITE_FLUSH_ACTION";
-  public static final String BUFFERWRITE_CLOSE_ACTION = "BUFFERWRITE_CLOSE_ACTION";
-  public static final String FILENODE_PROCESSOR_FLUSH_ACTION = "FILENODE_PROCESSOR_FLUSH_ACTION";
-
-  public static final String MREGE_EXTENSION = "merge";
-  public static final String ERR_EXTENSION = "err";
-  public static final String PATH_SEPARATOR = ".";
-  public static final String BUFFERWRITE_FILE_SEPARATOR = "-";
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriter.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriter.java
deleted file mode 100644
index a7542b5..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriter.java
+++ /dev/null
@@ -1,326 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.bufferwrite;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.FileSchema;
-import org.apache.iotdb.tsfile.write.writer.DefaultTsFileOutput;
-import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
-import org.apache.iotdb.tsfile.write.writer.TsFileOutput;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A tsfile io writer that has the ability to restore an incomplete tsfile. <br/> An incomplete
- * tsfile represents the file which does not have tsfile metadata in the end. Besides, the last
- * Chunk group data may be broken. This class can slice off the broken Chunk group data, accept
- * writing new data, and finally write the tsfile metadata. <br/> There are two cases: (1) though
- * the tsfile loses the tsfile metadata in the end, a corresponding. restore file exists. (2) no
- * .restore file, and then the class has to traverse all the data for fixing the file.
- */
-public class RestorableTsFileIOWriter extends TsFileIOWriter {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(RestorableTsFileIOWriter.class);
-
-  private static final int TS_METADATA_BYTE_SIZE = 4;
-  private static final int TS_POSITION_BYTE_SIZE = 8;
-
-  public static final String RESTORE_SUFFIX = ".restore";
-  private static final String DEFAULT_MODE = "rw";
-
-  private int lastFlushedChunkGroupIndex = 0;
-  /**
-   * chunk group metadata which are not serialized on disk (.restore file).
-   */
-  private List<ChunkGroupMetaData> append;
-
-  /**
-   * all chunk group metadata which have been serialized on disk.
-   */
-  private Map<String, Map<String, List<ChunkMetaData>>> metadatas;
-
-  /**
-   * unsealed data file.
-   */
-  private String insertFilePath;
-  /**
-   * corresponding index file.
-   */
-  private String restoreFilePath;
-
-  private boolean isNewResource = false;
-
-  public RestorableTsFileIOWriter(String processorName, String insertFilePath) throws IOException {
-    super();
-    this.insertFilePath = insertFilePath;
-    this.restoreFilePath = insertFilePath + RESTORE_SUFFIX;
-
-    this.metadatas = new HashMap<>();
-
-    File insertFile = new File(insertFilePath);
-    File restoreFile = new File(restoreFilePath);
-    if (insertFile.exists() && restoreFile.exists()) {
-      // read restore file
-      Pair<Long, List<ChunkGroupMetaData>> restoreInfo = readRestoreInfo();
-      long position = restoreInfo.left;
-      List<ChunkGroupMetaData> existedMetadatas = restoreInfo.right;
-      // cut off tsfile
-      this.out = new DefaultTsFileOutput(new FileOutputStream(insertFile, true));
-      out.truncate(position);
-      this.chunkGroupMetaDataList = existedMetadatas;
-      lastFlushedChunkGroupIndex = chunkGroupMetaDataList.size();
-      append = new ArrayList<>();
-      // recovery the metadata
-      recoverMetadata(existedMetadatas);
-      LOGGER.info(
-          "Recover the bufferwrite processor {}, the tsfile seriesPath is {}, "
-              + "the position of last flush is {}, the size of rowGroupMetadata is {}",
-          processorName, insertFilePath, position, existedMetadatas.size());
-      isNewResource = false;
-    } else {
-      try {
-        Files.deleteIfExists(insertFile.toPath());
-      } catch (IOException e) {
-        LOGGER.info("remove unsealed tsfile  failed: ", e);
-      }
-      try {
-        Files.deleteIfExists(restoreFile.toPath());
-      } catch (IOException e) {
-        LOGGER.info("remove unsealed tsfile restore file failed: ", e);
-      }
-      this.out = new DefaultTsFileOutput(new FileOutputStream(insertFile));
-      this.chunkGroupMetaDataList = new ArrayList<>();
-      lastFlushedChunkGroupIndex = chunkGroupMetaDataList.size();
-      append = new ArrayList<>();
-      startFile();
-      isNewResource = true;
-      writeRestoreInfo();
-    }
-
-  }
-
-  private void recoverMetadata(List<ChunkGroupMetaData> rowGroupMetaDatas) {
-    // TODO it is better if we can consider the problem caused by deletion
-    // and re-create time series here.
-    for (ChunkGroupMetaData rowGroupMetaData : rowGroupMetaDatas) {
-      String deviceId = rowGroupMetaData.getDeviceID();
-      if (!metadatas.containsKey(deviceId)) {
-        metadatas.put(deviceId, new HashMap<>());
-      }
-      for (ChunkMetaData chunkMetaData : rowGroupMetaData.getChunkMetaDataList()) {
-        String measurementId = chunkMetaData.getMeasurementUid();
-        if (!metadatas.get(deviceId).containsKey(measurementId)) {
-          metadatas.get(deviceId).put(measurementId, new ArrayList<>());
-        }
-        metadatas.get(deviceId).get(measurementId).add(chunkMetaData);
-      }
-    }
-  }
-
-  private void writeRestoreInfo() throws IOException {
-    long lastPosition = this.getPos();
-    // TODO: no need to create a TsRowGroupBlockMetadata, flush RowGroupMetadata one by one is ok
-    TsDeviceMetadata tsDeviceMetadata = new TsDeviceMetadata();
-    this.getAppendedRowGroupMetadata();
-    tsDeviceMetadata.setChunkGroupMetadataList(this.append);
-
-    try (RandomAccessFile out = new RandomAccessFile(restoreFilePath, DEFAULT_MODE)) {
-      if (out.length() > 0) {
-        out.seek(out.length() - TS_POSITION_BYTE_SIZE);
-      }
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      tsDeviceMetadata.serializeTo(baos);
-      // write metadata size using int
-      int metadataSize = baos.size();
-      out.write(BytesUtils.intToBytes(metadataSize));
-      // write metadata
-      out.write(baos.toByteArray());
-      // write tsfile position using byte[8] which is a long
-      byte[] lastPositionBytes = BytesUtils.longToBytes(lastPosition);
-      out.write(lastPositionBytes);
-    }
-  }
-
-  /**
-   * This is a private method. (It is default now for Unit Test only)
-   *
-   * @return a pair, whose left Long value is the tail position of the last complete Chunk Group in
-   * the unsealed file's position, and the right List value is the ChunkGroupMetadata of all
-   * complete Chunk Group in the same file.
-   * @throws IOException if errors when reading restoreFile.
-   */
-  Pair<Long, List<ChunkGroupMetaData>> readRestoreInfo() throws IOException {
-    byte[] lastPostionBytes = new byte[TS_POSITION_BYTE_SIZE];
-    List<ChunkGroupMetaData> groupMetaDatas = new ArrayList<>();
-    RandomAccessFile randomAccessFile = null;
-    randomAccessFile = new RandomAccessFile(restoreFilePath, DEFAULT_MODE);
-    try {
-      long fileLength = randomAccessFile.length();
-      // read tsfile position
-      long point = randomAccessFile.getFilePointer();
-      while (point + TS_POSITION_BYTE_SIZE < fileLength) {
-        byte[] metadataSizeBytes = new byte[TS_METADATA_BYTE_SIZE];
-        randomAccessFile.read(metadataSizeBytes);
-        int metadataSize = BytesUtils.bytesToInt(metadataSizeBytes);
-        byte[] thriftBytes = new byte[metadataSize];
-        randomAccessFile.read(thriftBytes);
-        ByteArrayInputStream inputStream = new ByteArrayInputStream(thriftBytes);
-        TsDeviceMetadata tsDeviceMetadata = TsDeviceMetadata.deserializeFrom(inputStream);
-        groupMetaDatas.addAll(tsDeviceMetadata.getChunkGroupMetaDataList());
-        point = randomAccessFile.getFilePointer();
-      }
-      // read the tsfile position information using byte[8] which is a long.
-      randomAccessFile.read(lastPostionBytes);
-      long lastPosition = BytesUtils.bytesToLong(lastPostionBytes);
-      return new Pair<>(lastPosition, groupMetaDatas);
-    } finally {
-      randomAccessFile.close();
-    }
-  }
-
-  /**
-   * get chunks' metadata from memory.
-   *
-   * @param deviceId the device id
-   * @param measurementId the sensor id
-   * @param dataType the value type
-   * @return chunks' metadata
-   */
-  public List<ChunkMetaData> getMetadatas(String deviceId, String measurementId, TSDataType dataType) {
-    List<ChunkMetaData> chunkMetaDatas = new ArrayList<>();
-    if (metadatas.containsKey(deviceId) && metadatas.get(deviceId).containsKey(measurementId)) {
-      for (ChunkMetaData chunkMetaData : metadatas.get(deviceId).get(measurementId)) {
-        // filter: if a device'sensor is defined as float type, and data has been persistent.
-        // Then someone deletes the timeseries and recreate it with Int type. We have to ignore
-        // all the stale data.
-        if (dataType.equals(chunkMetaData.getTsDataType())) {
-          chunkMetaDatas.add(chunkMetaData);
-        }
-      }
-    }
-    return chunkMetaDatas;
-  }
-
-  String getInsertFilePath() {
-    return insertFilePath;
-  }
-
-  public String getRestoreFilePath() {
-    return restoreFilePath;
-  }
-
-  boolean isNewResource() {
-    return isNewResource;
-  }
-
-  void setNewResource(boolean isNewResource) {
-    this.isNewResource = isNewResource;
-  }
-
-  public void flush() throws IOException {
-    writeRestoreInfo();
-  }
-
-  /**
-   * add all appendChunkGroupMetadatas into memory. After calling this method, other classes can
-   * read these metadata.
-   */
-  public void appendMetadata() {
-    if (!append.isEmpty()) {
-      for (ChunkGroupMetaData rowGroupMetaData : append) {
-        for (ChunkMetaData chunkMetaData : rowGroupMetaData.getChunkMetaDataList()) {
-          addInsertMetadata(rowGroupMetaData.getDeviceID(), chunkMetaData.getMeasurementUid(),
-              chunkMetaData);
-        }
-      }
-      append.clear();
-    }
-  }
-
-  private void addInsertMetadata(String deviceId, String measurementId,
-      ChunkMetaData chunkMetaData) {
-    if (!metadatas.containsKey(deviceId)) {
-      metadatas.put(deviceId, new HashMap<>());
-    }
-    if (!metadatas.get(deviceId).containsKey(measurementId)) {
-      metadatas.get(deviceId).put(measurementId, new ArrayList<>());
-    }
-    metadatas.get(deviceId).get(measurementId).add(chunkMetaData);
-  }
-
-  @Override
-  public void endFile(FileSchema schema) throws IOException {
-    super.endFile(schema);
-    try {
-      Files.delete(Paths.get(restoreFilePath));
-    } catch (IOException e) {
-      LOGGER.info("delete restore file {} failed, because {}", restoreFilePath, e.getMessage());
-    }
-  }
-
-  /**
-   * get all the chunkGroups' metadata which are appended after the last calling of this method, or
-   * after the class instance is initialized if this is the first time to call the method.
-   *
-   * @return a list of chunkgroup metadata
-   */
-  private List<ChunkGroupMetaData> getAppendedRowGroupMetadata() {
-    if (lastFlushedChunkGroupIndex < chunkGroupMetaDataList.size()) {
-      append.clear();
-      append.addAll(chunkGroupMetaDataList
-          .subList(lastFlushedChunkGroupIndex, chunkGroupMetaDataList.size()));
-      lastFlushedChunkGroupIndex = chunkGroupMetaDataList.size();
-    }
-    return append;
-  }
-
-  /**
-   * see {@link java.nio.channels.FileChannel#truncate(long)}.
-   */
-  public void truncate(long position) throws IOException {
-    out.truncate(position);
-  }
-
-  /**
-   * just for test.
-   *
-   * @return the output
-   */
-  TsFileOutput getOutput() {
-    return out;
-  }
-
-}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/RowGroupBlockMetaDataCache.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/DeviceMetaDataCache.java
similarity index 86%
rename from iotdb/src/main/java/org/apache/iotdb/db/engine/cache/RowGroupBlockMetaDataCache.java
rename to iotdb/src/main/java/org/apache/iotdb/db/engine/cache/DeviceMetaDataCache.java
index 044fb5d..994622a 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/RowGroupBlockMetaDataCache.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/DeviceMetaDataCache.java
@@ -29,11 +29,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * This class is used to cache <code>RowGroupBlockMetaDataCache</code> of tsfile in IoTDB.
+ * This class is used to cache <code>DeviceMetaDataCache</code> of tsfile in IoTDB.
  */
-public class RowGroupBlockMetaDataCache {
+public class DeviceMetaDataCache {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(RowGroupBlockMetaDataCache.class);
+  private static final Logger logger = LoggerFactory.getLogger(DeviceMetaDataCache.class);
 
   private static final int CACHE_SIZE = 100;
   /**
@@ -44,11 +44,11 @@ public class RowGroupBlockMetaDataCache {
   private AtomicLong cacheHintNum = new AtomicLong();
   private AtomicLong cacheRequestNum = new AtomicLong();
 
-  private RowGroupBlockMetaDataCache(int cacheSize) {
+  private DeviceMetaDataCache(int cacheSize) {
     lruCache = new LruLinkedHashMap(cacheSize, true);
   }
 
-  public static RowGroupBlockMetaDataCache getInstance() {
+  public static DeviceMetaDataCache getInstance() {
     return RowGroupBlockMetaDataCacheSingleton.INSTANCE;
   }
 
@@ -65,8 +65,8 @@ public class RowGroupBlockMetaDataCache {
       cacheRequestNum.incrementAndGet();
       if (lruCache.containsKey(jointPath)) {
         cacheHintNum.incrementAndGet();
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug(
+        if (logger.isDebugEnabled()) {
+          logger.debug(
               "Cache hint: the number of requests for cache is {}, "
                   + "the number of hints for cache is {}",
               cacheRequestNum.get(), cacheHintNum.get());
@@ -80,8 +80,8 @@ public class RowGroupBlockMetaDataCache {
           return lruCache.get(jointPath);
         }
       }
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("Cache didn't hint: the number of requests for cache is {}",
+      if (logger.isDebugEnabled()) {
+        logger.debug("Cache didn't hint: the number of requests for cache is {}",
             cacheRequestNum.get());
       }
       TsDeviceMetadata blockMetaData = TsFileMetadataUtils
@@ -108,8 +108,8 @@ public class RowGroupBlockMetaDataCache {
    */
   private static class RowGroupBlockMetaDataCacheSingleton {
 
-    private static final RowGroupBlockMetaDataCache INSTANCE = new
-        RowGroupBlockMetaDataCache(CACHE_SIZE);
+    private static final DeviceMetaDataCache INSTANCE = new
+        DeviceMetaDataCache(CACHE_SIZE);
   }
 
   /**
@@ -154,7 +154,7 @@ public class RowGroupBlockMetaDataCache {
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
-    RowGroupBlockMetaDataCache that = (RowGroupBlockMetaDataCache) o;
+    DeviceMetaDataCache that = (DeviceMetaDataCache) o;
     return Objects.equals(lruCache, that.lruCache) &&
             Objects.equals(cacheHintNum, that.cacheHintNum) &&
             Objects.equals(cacheRequestNum, that.cacheRequestNum);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
index 3cfe180..cff1a06 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
  */
 public class TsFileMetaDataCache {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TsFileMetaDataCache.class);
+  private static final Logger logger = LoggerFactory.getLogger(TsFileMetaDataCache.class);
   /**
    * key: The file seriesPath of tsfile.
    */
@@ -60,15 +60,15 @@ public class TsFileMetaDataCache {
         // read value from tsfile
         TsFileMetaData fileMetaData = TsFileMetadataUtils.getTsFileMetaData(path);
         cache.put(path, fileMetaData);
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug("Cache didn't hint: the number of requests for cache is {}",
+        if (logger.isDebugEnabled()) {
+          logger.debug("Cache didn't hint: the number of requests for cache is {}",
               cacheRequestNum.get());
         }
         return cache.get(path);
       } else {
         cacheHintNum.incrementAndGet();
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug(
+        if (logger.isDebugEnabled()) {
+          logger.debug(
               "Cache hint: the number of requests for cache is {}, the number of hints for cache "
                   + "is {}",
               cacheRequestNum.get(), cacheHintNum.get());
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetadataUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetadataUtils.java
index 4cd1602..ea63b37 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetadataUtils.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetadataUtils.java
@@ -41,14 +41,8 @@ public class TsFileMetadataUtils {
    * @return -meta data
    */
   public static TsFileMetaData getTsFileMetaData(String filePath) throws IOException {
-    TsFileSequenceReader reader = null;
-    try {
-      reader = new TsFileSequenceReader(filePath);
+    try (TsFileSequenceReader reader = new TsFileSequenceReader(filePath)) {
       return reader.readFileMetadata();
-    } finally {
-      if (reader != null) {
-        reader.close();
-      }
     }
   }
 
@@ -65,19 +59,13 @@ public class TsFileMetadataUtils {
     if (!fileMetaData.getDeviceMap().containsKey(deviceId)) {
       return null;
     } else {
-      TsFileSequenceReader reader = null;
-      try {
-        reader = new TsFileSequenceReader(filePath);
+      try (TsFileSequenceReader reader = new TsFileSequenceReader(filePath)) {
         long offset = fileMetaData.getDeviceMap().get(deviceId).getOffset();
         int size = fileMetaData.getDeviceMap().get(deviceId).getLen();
         ByteBuffer data = ByteBuffer.allocate(size);
         reader.readRaw(offset, size, data);
         data.flip();
         return TsDeviceMetadata.deserializeFrom(data);
-      } finally {
-        if (reader != null) {
-          reader.close();
-        }
       }
     }
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeFlushFuture.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeFlushFuture.java
deleted file mode 100644
index 85c5bb4..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeFlushFuture.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import org.apache.iotdb.db.utils.ImmediateFuture;
-
-public class FileNodeFlushFuture implements Future<Boolean> {
-  Future<Boolean> bufferWriteFlushFuture;
-  Future<Boolean> overflowFlushFuture;
-  boolean hasOverflowFlushTask;
-
-  public FileNodeFlushFuture(Future<Boolean> bufferWriteFlushFuture, Future<Boolean> overflowFlushFuture){
-    if(bufferWriteFlushFuture != null) {
-      this.bufferWriteFlushFuture = bufferWriteFlushFuture;
-    } else {
-      this.bufferWriteFlushFuture = new ImmediateFuture<>(true);
-    }
-    if(overflowFlushFuture !=null) {
-      this.overflowFlushFuture = overflowFlushFuture;
-      hasOverflowFlushTask = true;
-    } else {
-      this.overflowFlushFuture = new ImmediateFuture<>(true);
-      hasOverflowFlushTask = false;
-    }
-  }
-
-  /**
-   * @param mayInterruptIfRunning true if the thread executing this task should be interrupted;
-   * otherwise, in-progress tasks are allowed to complete
-   * @return true if both of the two sub-future are canceled successfully.
-   * @see Future#cancel(boolean) The difference is that this Future consists of two sub-Futures. If
-   * the first sub-future is canceled successfully but the second sub-future fails, the result is
-   * false.
-   */
-  @Override
-  public boolean cancel(boolean mayInterruptIfRunning) {
-    boolean bwResult = bufferWriteFlushFuture.cancel(mayInterruptIfRunning);
-    boolean ofResult = overflowFlushFuture.cancel(mayInterruptIfRunning);
-    return bwResult && ofResult;
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return bufferWriteFlushFuture.isCancelled() && overflowFlushFuture.isCancelled();
-  }
-
-  @Override
-  public boolean isDone() {
-    return bufferWriteFlushFuture.isDone() && overflowFlushFuture.isDone();
-  }
-
-  @Override
-  public Boolean get() throws InterruptedException, ExecutionException {
-    boolean bwResult = bufferWriteFlushFuture.get();
-    boolean ofResult = overflowFlushFuture.get();
-    return bwResult && ofResult;
-  }
-
-  @Override
-  public Boolean get(long timeout, TimeUnit unit)
-      throws InterruptedException, ExecutionException, TimeoutException {
-    boolean result = bufferWriteFlushFuture.get(timeout, unit);
-    result = result && overflowFlushFuture.get(timeout, unit);
-    return result;
-  }
-
-  public boolean isHasOverflowFlushTask() {
-    return hasOverflowFlushTask;
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java
deleted file mode 100644
index 4363cc3..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java
+++ /dev/null
@@ -1,1252 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.commons.io.FileUtils;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.directories.Directories;
-import org.apache.iotdb.db.engine.Processor;
-import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor;
-import org.apache.iotdb.db.engine.memcontrol.BasicMemController;
-import org.apache.iotdb.db.engine.overflow.io.OverflowProcessor;
-import org.apache.iotdb.db.engine.pool.FlushManager;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.exception.BufferWriteProcessorException;
-import org.apache.iotdb.db.exception.FileNodeManagerException;
-import org.apache.iotdb.db.exception.FileNodeProcessorException;
-import org.apache.iotdb.db.exception.MetadataErrorException;
-import org.apache.iotdb.db.exception.PathErrorException;
-import org.apache.iotdb.db.exception.ProcessorException;
-import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.monitor.IStatistic;
-import org.apache.iotdb.db.monitor.MonitorConstants;
-import org.apache.iotdb.db.monitor.StatMonitor;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.UpdatePlan;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.service.IService;
-import org.apache.iotdb.db.service.ServiceType;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
-import org.apache.iotdb.db.writelog.node.WriteLogNode;
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FileNodeManager implements IStatistic, IService {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(FileNodeManager.class);
-  private static final IoTDBConfig TsFileDBConf = IoTDBDescriptor.getInstance().getConfig();
-  private static final Directories directories = Directories.getInstance();
-  /**
-   * a folder that persist FileNodeProcessorStore classes. Each stroage group will have a subfolder.
-   * by default, it is system/info
-   */
-  private final String baseDir;
-
-  /**
-   * This map is used to manage all filenode processor,<br> the key is filenode name which is
-   * storage group seriesPath.
-   */
-  private ConcurrentHashMap<String, FileNodeProcessor> processorMap;
-  /**
-   * This set is used to store overflowed filenode name.<br> The overflowed filenode will be merge.
-   */
-  private volatile FileNodeManagerStatus fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-  // There is no need to add concurrently
-  private HashMap<String, AtomicLong> statParamsHashMap;
-
-  private FileNodeManager(String baseDir) {
-    processorMap = new ConcurrentHashMap<>();
-    statParamsHashMap = new HashMap<>();
-    //label: A
-    for (MonitorConstants.FileNodeManagerStatConstants fileNodeManagerStatConstant :
-        MonitorConstants.FileNodeManagerStatConstants.values()) {
-      statParamsHashMap.put(fileNodeManagerStatConstant.name(), new AtomicLong(0));
-    }
-
-    String normalizedBaseDir = baseDir;
-    if (normalizedBaseDir.charAt(normalizedBaseDir.length() - 1) != File.separatorChar) {
-      normalizedBaseDir += Character.toString(File.separatorChar);
-    }
-    this.baseDir = normalizedBaseDir;
-    File dir = new File(normalizedBaseDir);
-    if (dir.mkdirs()) {
-      LOGGER.info("{} dir home doesn't exist, create it", dir.getPath());
-    }
-    //TODO merge this with label A
-    if (TsFileDBConf.isEnableStatMonitor()) {
-      StatMonitor statMonitor = StatMonitor.getInstance();
-      registerStatMetadata();
-      statMonitor.registerStatistics(MonitorConstants.STAT_STORAGE_DELTA_NAME, this);
-    }
-  }
-
-  public static FileNodeManager getInstance() {
-    return FileNodeManagerHolder.INSTANCE;
-  }
-
-  private void updateStatHashMapWhenFail(TSRecord tsRecord) {
-    statParamsHashMap.get(MonitorConstants.FileNodeManagerStatConstants.TOTAL_REQ_FAIL.name())
-        .incrementAndGet();
-    statParamsHashMap.get(MonitorConstants.FileNodeManagerStatConstants.TOTAL_POINTS_FAIL.name())
-        .addAndGet(tsRecord.dataPointList.size());
-  }
-
-  /**
-   * get stats parameter hash map.
-   *
-   * @return the key represents the params' name, values is AtomicLong type
-   */
-  @Override
-  public Map<String, AtomicLong> getStatParamsHashMap() {
-    return statParamsHashMap;
-  }
-
-  @Override
-  public List<String> getAllPathForStatistic() {
-    List<String> list = new ArrayList<>();
-    for (MonitorConstants.FileNodeManagerStatConstants statConstant :
-        MonitorConstants.FileNodeManagerStatConstants.values()) {
-      list.add(MonitorConstants.STAT_STORAGE_DELTA_NAME + MonitorConstants.MONITOR_PATH_SEPARATOR
-          + statConstant.name());
-    }
-    return list;
-  }
-
-  @Override
-  public Map<String, TSRecord> getAllStatisticsValue() {
-    long curTime = System.currentTimeMillis();
-    TSRecord tsRecord = StatMonitor
-        .convertToTSRecord(getStatParamsHashMap(), MonitorConstants.STAT_STORAGE_DELTA_NAME,
-            curTime);
-    HashMap<String, TSRecord> ret = new HashMap<>();
-    ret.put(MonitorConstants.STAT_STORAGE_DELTA_NAME, tsRecord);
-    return ret;
-  }
-
-  /**
-   * Init Stat MetaDta.
-   */
-  @Override
-  public void registerStatMetadata() {
-    Map<String, String> hashMap = new HashMap<>();
-    for (MonitorConstants.FileNodeManagerStatConstants statConstant :
-        MonitorConstants.FileNodeManagerStatConstants.values()) {
-      hashMap
-          .put(MonitorConstants.STAT_STORAGE_DELTA_NAME + MonitorConstants.MONITOR_PATH_SEPARATOR
-              + statConstant.name(), MonitorConstants.DATA_TYPE_INT64);
-    }
-    StatMonitor.getInstance().registerStatStorageGroup(hashMap);
-  }
-
-  /**
-   * This function is just for unit test.
-   */
-  public synchronized void resetFileNodeManager() {
-    for (String key : statParamsHashMap.keySet()) {
-      statParamsHashMap.put(key, new AtomicLong());
-    }
-    processorMap.clear();
-  }
-
-  /**
-   * @param filenodeName storage name, e.g., root.a.b
-   */
-  private FileNodeProcessor constructNewProcessor(String filenodeName)
-      throws FileNodeManagerException {
-    try {
-      return new FileNodeProcessor(baseDir, filenodeName);
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error("Can't construct the FileNodeProcessor, the filenode is {}", filenodeName, e);
-      throw new FileNodeManagerException(e);
-    }
-  }
-
-  private FileNodeProcessor getProcessor(String path, boolean isWriteLock)
-      throws FileNodeManagerException {
-    String filenodeName;
-    try {
-      // return the stroage name
-      filenodeName = MManager.getInstance().getFileNameByPath(path);
-    } catch (PathErrorException e) {
-      LOGGER.error("MManager get filenode name error, seriesPath is {}", path);
-      throw new FileNodeManagerException(e);
-    }
-    FileNodeProcessor processor;
-    processor = processorMap.get(filenodeName);
-    if (processor != null) {
-      processor.lock(isWriteLock);
-    } else {
-      filenodeName = filenodeName.intern();
-      // calculate the value with same key synchronously
-      synchronized (filenodeName) {
-        processor = processorMap.get(filenodeName);
-        if (processor != null) {
-          processor.lock(isWriteLock);
-        } else {
-          // calculate the value with the key monitor
-          LOGGER.debug("construct a processor instance, the filenode is {}, Thread is {}",
-              filenodeName, Thread.currentThread().getId());
-          processor = constructNewProcessor(filenodeName);
-          processor.lock(isWriteLock);
-          processorMap.put(filenodeName, processor);
-        }
-      }
-    }
-    return processor;
-  }
-
-  /**
-   * recovery the filenode processor.
-   */
-  public void recovery() {
-    List<String> filenodeNames = null;
-    try {
-      filenodeNames = MManager.getInstance().getAllFileNames();
-    } catch (MetadataErrorException e) {
-      LOGGER.error("Restoring all FileNodes failed.", e);
-      return;
-    }
-    for (String filenodeName : filenodeNames) {
-      FileNodeProcessor fileNodeProcessor = null;
-      try {
-        fileNodeProcessor = getProcessor(filenodeName, true);
-        if (fileNodeProcessor.shouldRecovery()) {
-          LOGGER.info("Recovery the filenode processor, the filenode is {}, the status is {}",
-              filenodeName, fileNodeProcessor.getFileNodeProcessorStatus());
-          fileNodeProcessor.fileNodeRecovery();
-        }
-      } catch (FileNodeManagerException | FileNodeProcessorException e) {
-        LOGGER.error("Restoring fileNode {} failed.", filenodeName, e);
-      } finally {
-        if (fileNodeProcessor != null) {
-          fileNodeProcessor.writeUnlock();
-        }
-      }
-      // add index check sum
-    }
-  }
-
-  /**
-   * insert TsRecord into storage group.
-   *
-   * @param tsRecord input Data
-   * @param isMonitor if true, the insertion is done by StatMonitor and the statistic Info will not
-   * be recorded. if false, the statParamsHashMap will be updated.
-   * @return an int value represents the insert type
-   */
-  public int insert(TSRecord tsRecord, boolean isMonitor) throws FileNodeManagerException {
-    long timestamp = tsRecord.time;
-
-    String deviceId = tsRecord.deviceId;
-    checkTimestamp(tsRecord);
-    updateStat(isMonitor, tsRecord);
-
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    int insertType;
-
-    try {
-      long lastUpdateTime = fileNodeProcessor.getFlushLastUpdateTime(deviceId);
-      if (timestamp < lastUpdateTime) {
-        insertOverflow(fileNodeProcessor, timestamp, tsRecord, isMonitor, deviceId);
-        insertType = 1;
-      } else {
-        insertBufferWrite(fileNodeProcessor, timestamp, isMonitor, tsRecord, deviceId);
-        insertType = 2;
-      }
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error(String.format("Encounter an error when closing the buffer write processor %s.",
-          fileNodeProcessor.getProcessorName()), e);
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-    // Modify the insert
-    if (!isMonitor) {
-      fileNodeProcessor.getStatParamsHashMap()
-          .get(MonitorConstants.FileNodeProcessorStatConstants.TOTAL_POINTS_SUCCESS.name())
-          .addAndGet(tsRecord.dataPointList.size());
-      fileNodeProcessor.getStatParamsHashMap()
-          .get(MonitorConstants.FileNodeProcessorStatConstants.TOTAL_REQ_SUCCESS.name())
-          .incrementAndGet();
-      statParamsHashMap.get(MonitorConstants.FileNodeManagerStatConstants.TOTAL_REQ_SUCCESS.name())
-          .incrementAndGet();
-      statParamsHashMap
-          .get(MonitorConstants.FileNodeManagerStatConstants.TOTAL_POINTS_SUCCESS.name())
-          .addAndGet(tsRecord.dataPointList.size());
-    }
-    return insertType;
-  }
-
-  private void writeLog(TSRecord tsRecord, boolean isMonitor, WriteLogNode logNode)
-      throws FileNodeManagerException {
-    try {
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        String[] measurementList = new String[tsRecord.dataPointList.size()];
-        String[] insertValues = new String[tsRecord.dataPointList.size()];
-        int i=0;
-        for (DataPoint dp : tsRecord.dataPointList) {
-          measurementList[i] = dp.getMeasurementId();
-          insertValues[i] = dp.getValue().toString();
-          i++;
-        }
-        logNode.write(new InsertPlan(2, tsRecord.deviceId, tsRecord.time, measurementList,
-            insertValues));
-      }
-    } catch (IOException e) {
-      if (!isMonitor) {
-        updateStatHashMapWhenFail(tsRecord);
-      }
-      throw new FileNodeManagerException(e);
-    }
-  }
-
-  private void checkTimestamp(TSRecord tsRecord) throws FileNodeManagerException {
-    if (tsRecord.time < 0) {
-      LOGGER.error("The insert time lt 0, {}.", tsRecord);
-      throw new FileNodeManagerException("The insert time lt 0, the tsrecord is " + tsRecord);
-    }
-  }
-
-  private void updateStat(boolean isMonitor, TSRecord tsRecord) {
-    if (!isMonitor) {
-      statParamsHashMap.get(MonitorConstants.FileNodeManagerStatConstants.TOTAL_POINTS.name())
-          .addAndGet(tsRecord.dataPointList.size());
-    }
-  }
-
-  private void insertOverflow(FileNodeProcessor fileNodeProcessor, long timestamp,
-      TSRecord tsRecord, boolean isMonitor, String deviceId)
-      throws FileNodeManagerException {
-    // get overflow processor
-    OverflowProcessor overflowProcessor;
-    String filenodeName = fileNodeProcessor.getProcessorName();
-    try {
-      overflowProcessor = fileNodeProcessor.getOverflowProcessor(filenodeName);
-    } catch (IOException e) {
-      LOGGER.error("Get the overflow processor failed, the filenode is {}, insert time is {}",
-          filenodeName, timestamp);
-      if (!isMonitor) {
-        updateStatHashMapWhenFail(tsRecord);
-      }
-      throw new FileNodeManagerException(e);
-    }
-    // write wal
-    try {
-      writeLog(tsRecord, isMonitor, overflowProcessor.getLogNode());
-    } catch (IOException e) {
-      throw new FileNodeManagerException(e);
-    }
-    // write overflow data
-    try {
-      overflowProcessor.insert(tsRecord);
-      fileNodeProcessor.changeTypeToChanged(deviceId, timestamp);
-      fileNodeProcessor.setOverflowed(true);
-    } catch (IOException e) {
-      LOGGER.error("Insert into overflow error, the reason is {}", e);
-      if (!isMonitor) {
-        updateStatHashMapWhenFail(tsRecord);
-      }
-      throw new FileNodeManagerException(e);
-    }
-  }
-
-  private void insertBufferWrite(FileNodeProcessor fileNodeProcessor, long timestamp,
-      boolean isMonitor, TSRecord tsRecord, String deviceId)
-      throws FileNodeManagerException, FileNodeProcessorException {
-    // get bufferwrite processor
-    BufferWriteProcessor bufferWriteProcessor;
-    String filenodeName = fileNodeProcessor.getProcessorName();
-    try {
-      bufferWriteProcessor = fileNodeProcessor.getBufferWriteProcessor(filenodeName, timestamp);
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error("Get the bufferwrite processor failed, the filenode is {}, insert time is {}",
-          filenodeName, timestamp);
-      if (!isMonitor) {
-        updateStatHashMapWhenFail(tsRecord);
-      }
-      throw new FileNodeManagerException(e);
-    }
-    // Add a new interval file to newfilelist
-    if (bufferWriteProcessor.isNewProcessor()) {
-      bufferWriteProcessor.setNewProcessor(false);
-      String bufferwriteBaseDir = bufferWriteProcessor.getBaseDir();
-      String bufferwriteRelativePath = bufferWriteProcessor.getFileRelativePath();
-      try {
-        fileNodeProcessor.addIntervalFileNode(new File(new File(bufferwriteBaseDir), bufferwriteRelativePath));
-      } catch (Exception e) {
-        if (!isMonitor) {
-          updateStatHashMapWhenFail(tsRecord);
-        }
-        throw new FileNodeManagerException(e);
-      }
-    }
-    // write wal
-    try {
-      writeLog(tsRecord, isMonitor, bufferWriteProcessor.getLogNode());
-    } catch (IOException e) {
-      throw new FileNodeManagerException(e);
-    }
-    // Write data
-    long prevStartTime = fileNodeProcessor.getIntervalFileNodeStartTime(deviceId);
-    long prevUpdateTime = fileNodeProcessor.getLastUpdateTime(deviceId);
-
-    fileNodeProcessor.setIntervalFileNodeStartTime(deviceId);
-    fileNodeProcessor.setLastUpdateTime(deviceId, timestamp);
-    try {
-      if (!bufferWriteProcessor.write(tsRecord)) {
-        // undo time update
-        fileNodeProcessor.setIntervalFileNodeStartTime(deviceId, prevStartTime);
-        fileNodeProcessor.setLastUpdateTime(deviceId, prevUpdateTime);
-      }
-    } catch (BufferWriteProcessorException e) {
-      if (!isMonitor) {
-        updateStatHashMapWhenFail(tsRecord);
-      }
-      throw new FileNodeManagerException(e);
-    }
-
-    if (bufferWriteProcessor
-        .getFileSize() > IoTDBDescriptor.getInstance()
-        .getConfig().getBufferwriteFileSizeThreshold()) {
-      if (LOGGER.isInfoEnabled()) {
-        LOGGER.info(
-            "The filenode processor {} will close the bufferwrite processor, "
-                + "because the size[{}] of tsfile {} reaches the threshold {}",
-            filenodeName, MemUtils.bytesCntToStr(bufferWriteProcessor.getFileSize()),
-            bufferWriteProcessor.getInsertFilePath(), MemUtils.bytesCntToStr(
-                IoTDBDescriptor.getInstance().getConfig().getBufferwriteFileSizeThreshold()));
-      }
-
-      fileNodeProcessor.closeBufferWrite();
-    }
-  }
-
-  /**
-   * update data.
-   */
-  public void update(String deviceId, String measurementId, long startTime, long endTime,
-      TSDataType type, String v)
-      throws FileNodeManagerException {
-
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-
-      long lastUpdateTime = fileNodeProcessor.getLastUpdateTime(deviceId);
-      if (startTime > lastUpdateTime) {
-        LOGGER.warn("The update range is error, startTime {} is great than lastUpdateTime {}",
-            startTime,
-            lastUpdateTime);
-        return;
-      }
-      long finalEndTime = endTime > lastUpdateTime ? lastUpdateTime : endTime;
-
-      String filenodeName = fileNodeProcessor.getProcessorName();
-      // get overflow processor
-      OverflowProcessor overflowProcessor;
-      try {
-        overflowProcessor = fileNodeProcessor.getOverflowProcessor(filenodeName);
-      } catch (IOException e) {
-        LOGGER.error(
-            "Get the overflow processor failed, the filenode is {}, "
-                + "insert time range is from {} to {}",
-            filenodeName, startTime, finalEndTime);
-        throw new FileNodeManagerException(e);
-      }
-      overflowProcessor.update(deviceId, measurementId, startTime, finalEndTime, type, v);
-      // change the type of tsfile to overflowed
-      fileNodeProcessor.changeTypeToChanged(deviceId, startTime, finalEndTime);
-      fileNodeProcessor.setOverflowed(true);
-
-      // write wal
-      try {
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-          overflowProcessor.getLogNode().write(
-              new UpdatePlan(startTime, finalEndTime, v, new Path(deviceId
-                  + "." + measurementId)));
-        }
-      } catch (IOException e) {
-        throw new FileNodeManagerException(e);
-      }
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-  /**
-   * delete data.
-   */
-  public void delete(String deviceId, String measurementId, long timestamp)
-      throws FileNodeManagerException {
-
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-      long lastUpdateTime = fileNodeProcessor.getLastUpdateTime(deviceId);
-      // no tsfile data, the delete operation is invalid
-      if (lastUpdateTime == -1) {
-        LOGGER.warn("The last update time is -1, delete overflow is invalid, "
-                + "the filenode processor is {}",
-            fileNodeProcessor.getProcessorName());
-      } else {
-        // write wal
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-          // get processors for wal
-          String filenodeName = fileNodeProcessor.getProcessorName();
-          OverflowProcessor overflowProcessor;
-          BufferWriteProcessor bufferWriteProcessor;
-          try {
-            overflowProcessor = fileNodeProcessor.getOverflowProcessor(filenodeName);
-            // in case that no BufferWriteProcessor is available, a new BufferWriteProcessor is
-            // needed to access LogNode.
-            // TODO this may make the time range of the next TsFile a little wider
-            bufferWriteProcessor = fileNodeProcessor.getBufferWriteProcessor(filenodeName,
-                lastUpdateTime + 1);
-          } catch (IOException | FileNodeProcessorException e) {
-            LOGGER.error("Getting the processor failed, the filenode is {}, delete time is {}.",
-                filenodeName, timestamp);
-            throw new FileNodeManagerException(e);
-          }
-          try {
-            overflowProcessor.getLogNode().write(new DeletePlan(timestamp,
-                new Path(deviceId + "." + measurementId)));
-            bufferWriteProcessor.getLogNode().write(new DeletePlan(timestamp,
-                new Path(deviceId + "." + measurementId)));
-          } catch (IOException e) {
-            throw new FileNodeManagerException(e);
-          }
-        }
-
-        try {
-          fileNodeProcessor.delete(deviceId, measurementId, timestamp);
-        } catch (IOException e) {
-          throw new FileNodeManagerException(e);
-        }
-        // change the type of tsfile to overflowed
-        fileNodeProcessor.changeTypeToChangedForDelete(deviceId, timestamp);
-        fileNodeProcessor.setOverflowed(true);
-
-      }
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-  private void delete(String processorName,
-      Iterator<Map.Entry<String, FileNodeProcessor>> processorIterator)
-      throws FileNodeManagerException {
-    if (!processorMap.containsKey(processorName)) {
-      //TODO do we need to call processorIterator.remove() ?
-      LOGGER.warn("The processorMap doesn't contain the filenode processor {}.", processorName);
-      return;
-    }
-    LOGGER.info("Try to delete the filenode processor {}.", processorName);
-    FileNodeProcessor processor = processorMap.get(processorName);
-    if (!processor.tryWriteLock()) {
-      throw new FileNodeManagerException(String
-          .format("Can't delete the filenode processor %s because Can't get the write lock.",
-              processorName));
-    }
-
-    try {
-      if (!processor.canBeClosed()) {
-        LOGGER.warn("The filenode processor {} can't be deleted.", processorName);
-        return;
-      }
-
-      try {
-        LOGGER.info("Delete the filenode processor {}.", processorName);
-        processor.delete();
-        processorIterator.remove();
-      } catch (ProcessorException e) {
-        LOGGER.error("Delete the filenode processor {} by iterator error.", processorName, e);
-        throw new FileNodeManagerException(e);
-      }
-    } finally {
-      processor.writeUnlock();
-    }
-  }
-
-  /**
-   * Similar to delete(), but only deletes data in BufferWrite. Only used by WAL recovery.
-   */
-  public void deleteBufferWrite(String deviceId, String measurementId, long timestamp)
-      throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-      fileNodeProcessor.deleteBufferWrite(deviceId, measurementId, timestamp);
-    } catch (BufferWriteProcessorException | IOException e) {
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-    // change the type of tsfile to overflowed
-    fileNodeProcessor.changeTypeToChangedForDelete(deviceId, timestamp);
-    fileNodeProcessor.setOverflowed(true);
-  }
-
-  /**
-   * Similar to delete(), but only deletes data in Overflow. Only used by WAL recovery.
-   */
-  public void deleteOverflow(String deviceId, String measurementId, long timestamp)
-      throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-      fileNodeProcessor.deleteOverflow(deviceId, measurementId, timestamp);
-    } catch (IOException e) {
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-    // change the type of tsfile to overflowed
-    fileNodeProcessor.changeTypeToChangedForDelete(deviceId, timestamp);
-    fileNodeProcessor.setOverflowed(true);
-  }
-
-  /**
-   * begin query.
-   *
-   * @param deviceId queried deviceId
-   * @return a query token for the device.
-   */
-  public int beginQuery(String deviceId) throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-      LOGGER.debug("Get the FileNodeProcessor: filenode is {}, begin query.",
-          fileNodeProcessor.getProcessorName());
-      return fileNodeProcessor.addMultiPassCount();
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-  /**
-   * query data.
-   */
-  public QueryDataSource query(SingleSeriesExpression seriesExpression, QueryContext context)
-      throws FileNodeManagerException {
-    String deviceId = seriesExpression.getSeriesPath().getDevice();
-    String measurementId = seriesExpression.getSeriesPath().getMeasurement();
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, false);
-    LOGGER.debug("Get the FileNodeProcessor: filenode is {}, query.",
-        fileNodeProcessor.getProcessorName());
-    try {
-      QueryDataSource queryDataSource;
-      // query operation must have overflow processor
-      if (!fileNodeProcessor.hasOverflowProcessor()) {
-        try {
-          fileNodeProcessor.getOverflowProcessor(fileNodeProcessor.getProcessorName());
-        } catch (IOException e) {
-          LOGGER.error("Get the overflow processor failed, the filenode is {}, query is {},{}",
-              fileNodeProcessor.getProcessorName(), deviceId, measurementId);
-          throw new FileNodeManagerException(e);
-        }
-      }
-      try {
-        queryDataSource = fileNodeProcessor.query(deviceId, measurementId, context);
-      } catch (FileNodeProcessorException e) {
-        LOGGER.error("Query error: the deviceId {}, the measurementId {}", deviceId, measurementId,
-            e);
-        throw new FileNodeManagerException(e);
-      }
-      // return query structure
-      return queryDataSource;
-    } finally {
-      fileNodeProcessor.readUnlock();
-    }
-  }
-
-  /**
-   * end query.
-   */
-  public void endQuery(String deviceId, int token) throws FileNodeManagerException {
-
-    FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true);
-    try {
-      LOGGER.debug("Get the FileNodeProcessor: {} end query.",
-          fileNodeProcessor.getProcessorName());
-      fileNodeProcessor.decreaseMultiPassCount(token);
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error("Failed to end query: the deviceId {}, token {}.", deviceId, token, e);
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-  /**
-   * Append one specified tsfile to the storage group. <b>This method is only provided for
-   * transmission module</b>
-   *
-   * @param fileNodeName the seriesPath of storage group
-   * @param appendFile the appended tsfile information
-   */
-  public boolean appendFileToFileNode(String fileNodeName, TsFileResource appendFile,
-      String appendFilePath) throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(fileNodeName, true);
-    try {
-      // check append file
-      for (Map.Entry<String, Long> entry : appendFile.getStartTimeMap().entrySet()) {
-        if (fileNodeProcessor.getLastUpdateTime(entry.getKey()) >= entry.getValue()) {
-          return false;
-        }
-      }
-      // close bufferwrite file
-      fileNodeProcessor.closeBufferWrite();
-      // append file to storage group.
-      fileNodeProcessor.appendFile(appendFile, appendFilePath);
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error("Cannot append the file {} to {}", appendFile.getFile().getAbsolutePath(), fileNodeName, e);
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-    return true;
-  }
-
-  /**
-   * get all overlap tsfiles which are conflict with the appendFile.
-   *
-   * @param fileNodeName the seriesPath of storage group
-   * @param appendFile the appended tsfile information
-   */
-  public List<String> getOverlapFilesFromFileNode(String fileNodeName, TsFileResource appendFile,
-      String uuid) throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(fileNodeName, true);
-    List<String> overlapFiles;
-    try {
-      overlapFiles = fileNodeProcessor.getOverlapFiles(appendFile, uuid);
-    } catch (FileNodeProcessorException e) {
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-    return overlapFiles;
-  }
-
-  /**
-   * merge all overflowed filenode.
-   *
-   * @throws FileNodeManagerException FileNodeManagerException
-   */
-  public void mergeAll() throws FileNodeManagerException {
-    if (fileNodeManagerStatus != FileNodeManagerStatus.NONE) {
-      LOGGER.warn("Failed to merge all overflowed filenode, because filenode manager status is {}",
-          fileNodeManagerStatus);
-      return;
-    }
-
-    fileNodeManagerStatus = FileNodeManagerStatus.MERGE;
-    LOGGER.info("Start to merge all overflowed filenode");
-    List<String> allFileNodeNames;
-    try {
-      allFileNodeNames = MManager.getInstance().getAllFileNames();
-    } catch (MetadataErrorException e) {
-      LOGGER.error("Get all storage group seriesPath error,", e);
-      throw new FileNodeManagerException(e);
-    }
-    List<Future<?>> futureTasks = new ArrayList<>();
-    for (String fileNodeName : allFileNodeNames) {
-      FileNodeProcessor fileNodeProcessor = getProcessor(fileNodeName, true);
-      try {
-        Future<?> task = fileNodeProcessor.submitToMerge();
-        if (task != null) {
-          LOGGER.info("Submit the filenode {} to the merge pool", fileNodeName);
-          futureTasks.add(task);
-        }
-      } finally {
-        fileNodeProcessor.writeUnlock();
-      }
-    }
-    long totalTime = 0;
-    // loop waiting for merge to end, the longest waiting time is
-    // 60s.
-    int time = 2;
-    List<Exception> mergeException = new ArrayList<>();
-    for (Future<?> task : futureTasks) {
-      while (!task.isDone()) {
-        try {
-          LOGGER.info(
-              "Waiting for the end of merge, already waiting for {}s, "
-                  + "continue to wait anothor {}s",
-              totalTime, time);
-          TimeUnit.SECONDS.sleep(time);
-          totalTime += time;
-          time = updateWaitTime(time);
-        } catch (InterruptedException e) {
-          LOGGER.error("Unexpected interruption {}", e);
-          Thread.currentThread().interrupt();
-        }
-      }
-      try {
-        task.get();
-      } catch (InterruptedException e) {
-        LOGGER.error("Unexpected interruption {}", e);
-      } catch (ExecutionException e) {
-        mergeException.add(e);
-        LOGGER.error("The exception for merge: {}", e);
-      }
-    }
-    if (!mergeException.isEmpty()) {
-      // just throw the first exception
-      throw new FileNodeManagerException(mergeException.get(0));
-    }
-    fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-    LOGGER.info("End to merge all overflowed filenode");
-  }
-
-  private int updateWaitTime(int time) {
-    return time < 32 ? time * 2 : 60;
-  }
-
-  /**
-   * try to close the filenode processor. The name of filenode processor is processorName
-   */
-  private boolean closeOneProcessor(String processorName) throws FileNodeManagerException {
-    if (!processorMap.containsKey(processorName)) {
-      return true;
-    }
-
-    Processor processor = processorMap.get(processorName);
-    if (processor.tryWriteLock()) {
-      try {
-        if (processor.canBeClosed()) {
-          processor.close();
-          return true;
-        } else {
-          return false;
-        }
-      } catch (ProcessorException e) {
-        LOGGER.error("Close the filenode processor {} error.", processorName, e);
-        throw new FileNodeManagerException(e);
-      } finally {
-        processor.writeUnlock();
-      }
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * delete one filenode.
-   */
-  public void deleteOneFileNode(String processorName) throws FileNodeManagerException {
-    if (fileNodeManagerStatus != FileNodeManagerStatus.NONE) {
-      return;
-    }
-
-    fileNodeManagerStatus = FileNodeManagerStatus.CLOSE;
-    try {
-      if (processorMap.containsKey(processorName)) {
-        deleteFileNodeBlocked(processorName);
-      }
-      String fileNodePath = TsFileDBConf.getFileNodeDir();
-      fileNodePath = standardizeDir(fileNodePath) + processorName;
-      FileUtils.deleteDirectory(new File(fileNodePath));
-
-      cleanBufferWrite(processorName);
-      cleanOverflow(processorName);
-
-      MultiFileLogNodeManager.getInstance()
-          .deleteNode(processorName + IoTDBConstant.BUFFERWRITE_LOG_NODE_SUFFIX);
-      MultiFileLogNodeManager.getInstance()
-          .deleteNode(processorName + IoTDBConstant.OVERFLOW_LOG_NODE_SUFFIX);
-    } catch (IOException e) {
-      LOGGER.error("Delete the filenode processor {} error.", processorName, e);
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-    }
-  }
-
-  private void cleanOverflow(String processorName) throws IOException {
-    String overflowDirPath = TsFileDBConf.getOverflowDataDir();
-    String overflowPath = standardizeDir(overflowDirPath) + processorName;
-    File overflowDir = new File(overflowPath);
-    // free and close the streams under this overflow directory
-    if (!overflowDir.exists()) {
-      return;
-    }
-    File[] overflowSubDirs = overflowDir.listFiles();
-    if (overflowSubDirs == null) {
-      return;
-    }
-    for (File overflowSubDir : overflowSubDirs) {
-      closeAndRemoveReader(overflowSubDir);
-    }
-    FileUtils.deleteDirectory(overflowDir);
-  }
-
-  private void cleanBufferWrite(String processorName) throws IOException {
-    List<String> bufferwritePathList = directories.getAllTsFileFolders();
-    for (String bufferwritePath : bufferwritePathList) {
-      bufferwritePath = standardizeDir(bufferwritePath) + processorName;
-      File bufferDir = new File(bufferwritePath);
-      // free and close the streams under this bufferwrite directory
-      if (!bufferDir.exists()) {
-        continue;
-      }
-      closeAndRemoveReader(bufferDir);
-      FileUtils.deleteDirectory(new File(bufferwritePath));
-    }
-  }
-
-  private void closeAndRemoveReader(File folder) throws IOException {
-    File[] files = folder.listFiles();
-    if (files != null) {
-      for (File file : files) {
-        FileReaderManager.getInstance().closeFileAndRemoveReader(file.getPath());
-      }
-    }
-  }
-
-  private void deleteFileNodeBlocked(String processorName) throws FileNodeManagerException {
-    LOGGER.info("Forced to delete the filenode processor {}", processorName);
-    FileNodeProcessor processor = processorMap.get(processorName);
-    while (true) {
-      if (processor.tryWriteLock()) {
-        try {
-          if (processor.canBeClosed()) {
-            LOGGER.info("Delete the filenode processor {}.", processorName);
-            processor.delete();
-            processorMap.remove(processorName);
-            break;
-          } else {
-            LOGGER.info(
-                "Can't delete the filenode processor {}, "
-                    + "because the filenode processor can't be closed."
-                    + " Wait 100ms to retry");
-          }
-        } catch (ProcessorException e) {
-          LOGGER.error("Delete the filenode processor {} error.", processorName, e);
-          throw new FileNodeManagerException(e);
-        } finally {
-          processor.writeUnlock();
-        }
-      } else {
-        LOGGER.info(
-            "Can't delete the filenode processor {}, because it can't get the write lock."
-                + " Wait 100ms to retry", processorName);
-      }
-      try {
-        TimeUnit.MILLISECONDS.sleep(100);
-      } catch (InterruptedException e) {
-        LOGGER.error(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
-  private String standardizeDir(String originalPath) {
-    String res = originalPath;
-    if ((originalPath.length() > 0
-        && originalPath.charAt(originalPath.length() - 1) != File.separatorChar)
-        || originalPath.length() == 0) {
-      res = originalPath + File.separatorChar;
-    }
-    return res;
-  }
-
-  /**
-   * add time series.
-   */
-  public void addTimeSeries(Path path, TSDataType dataType, TSEncoding encoding,
-      CompressionType compressor,
-      Map<String, String> props) throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(path.getFullPath(), true);
-    try {
-      fileNodeProcessor.addTimeSeries(path.getMeasurement(), dataType, encoding, compressor, props);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-
-  /**
-   * Force to close the filenode processor.
-   */
-  public void closeOneFileNode(String processorName) throws FileNodeManagerException {
-    if (fileNodeManagerStatus != FileNodeManagerStatus.NONE) {
-      return;
-    }
-
-    fileNodeManagerStatus = FileNodeManagerStatus.CLOSE;
-    try {
-      LOGGER.info("Force to close the filenode processor {}.", processorName);
-      while (!closeOneProcessor(processorName)) {
-        try {
-          LOGGER.info("Can't force to close the filenode processor {}, wait 100ms to retry",
-              processorName);
-          TimeUnit.MILLISECONDS.sleep(100);
-        } catch (InterruptedException e) {
-          // ignore the interrupted exception
-          LOGGER.error("Unexpected interruption {}", e);
-          Thread.currentThread().interrupt();
-        }
-      }
-    } finally {
-      fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-    }
-  }
-
-  /**
-   * try to close the filenode processor.
-   */
-  private void close(String processorName) throws FileNodeManagerException {
-    if (!processorMap.containsKey(processorName)) {
-      LOGGER.warn("The processorMap doesn't contain the filenode processor {}.", processorName);
-      return;
-    }
-    LOGGER.info("Try to close the filenode processor {}.", processorName);
-    FileNodeProcessor processor = processorMap.get(processorName);
-    if (!processor.tryWriteLock()) {
-      LOGGER.warn("Can't get the write lock of the filenode processor {}.", processorName);
-      return;
-    }
-    try {
-      if (processor.canBeClosed()) {
-        try {
-          LOGGER.info("Close the filenode processor {}.", processorName);
-          processor.close();
-        } catch (ProcessorException e) {
-          LOGGER.error("Close the filenode processor {} error.", processorName, e);
-          throw new FileNodeManagerException(e);
-        }
-      } else {
-        LOGGER.warn("The filenode processor {} can't be closed.", processorName);
-      }
-    } finally {
-      processor.writeUnlock();
-    }
-  }
-
-  /**
-   * delete all filenode.
-   */
-  public synchronized boolean deleteAll() throws FileNodeManagerException {
-    LOGGER.info("Start deleting all filenode");
-    if (fileNodeManagerStatus != FileNodeManagerStatus.NONE) {
-      LOGGER.info("Failed to delete all filenode processor because of merge operation");
-      return false;
-    }
-
-    fileNodeManagerStatus = FileNodeManagerStatus.CLOSE;
-    try {
-      Iterator<Map.Entry<String, FileNodeProcessor>> processorIterator = processorMap.entrySet()
-          .iterator();
-      while (processorIterator.hasNext()) {
-        Map.Entry<String, FileNodeProcessor> processorEntry = processorIterator.next();
-        delete(processorEntry.getKey(), processorIterator);
-      }
-      return processorMap.isEmpty();
-    } finally {
-      LOGGER.info("Deleting all FileNodeProcessors ends");
-      fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-    }
-  }
-
-  /**
-   * Try to close All.
-   */
-  public void closeAll() throws FileNodeManagerException {
-    LOGGER.info("Start closing all filenode processor");
-    if (fileNodeManagerStatus != FileNodeManagerStatus.NONE) {
-      LOGGER.info("Failed to close all filenode processor because of merge operation");
-      return;
-    }
-    fileNodeManagerStatus = FileNodeManagerStatus.CLOSE;
-    try {
-      for (Map.Entry<String, FileNodeProcessor> processorEntry : processorMap.entrySet()) {
-        close(processorEntry.getKey());
-      }
-    } finally {
-      LOGGER.info("Close all FileNodeProcessors ends");
-      fileNodeManagerStatus = FileNodeManagerStatus.NONE;
-    }
-  }
-
-  /**
-   * force flush to control memory usage.
-   */
-  public void forceFlush(BasicMemController.UsageLevel level) {
-    // you may add some delicate process like below
-    // or you could provide multiple methods for different urgency
-    switch (level) {
-      // only select the most urgent (most active or biggest in size)
-      // processors to flush
-      // only select top 10% active memory user to flush
-      case WARNING:
-        try {
-          flushTop(0.1f);
-        } catch (IOException e) {
-          LOGGER.error("force flush memory data error: {}", e);
-        }
-        break;
-      // force all processors to flush
-      case DANGEROUS:
-        try {
-          flushAll();
-        } catch (IOException e) {
-          LOGGER.error("force flush memory data error: {}", e);
-        }
-        break;
-      // if the flush thread pool is not full ( or half full), start a new
-      // flush task
-      case SAFE:
-        if (FlushManager.getInstance().getActiveCnt() < 0.5 * FlushManager.getInstance()
-            .getThreadCnt()) {
-          try {
-            flushTop(0.01f);
-          } catch (IOException e) {
-            LOGGER.error("force flush memory data error: ", e);
-          }
-        }
-        break;
-      default:
-    }
-  }
-
-  private void flushAll() throws IOException {
-    for (FileNodeProcessor processor : processorMap.values()) {
-      if (!processor.tryLock(true)) {
-        continue;
-      }
-      try {
-        boolean isMerge = processor.flush().isHasOverflowFlushTask();
-        if (isMerge) {
-          processor.submitToMerge();
-        }
-      } finally {
-        processor.unlock(true);
-      }
-    }
-  }
-
-  private void flushTop(float percentage) throws IOException {
-    List<FileNodeProcessor> tempProcessors = new ArrayList<>(processorMap.values());
-    // sort the tempProcessors as descending order
-    tempProcessors.sort((o1, o2) -> (int) (o2.memoryUsage() - o1.memoryUsage()));
-    int flushNum =
-        (int) (tempProcessors.size() * percentage) > 1
-            ? (int) (tempProcessors.size() * percentage)
-            : 1;
-    for (int i = 0; i < flushNum && i < tempProcessors.size(); i++) {
-      FileNodeProcessor processor = tempProcessors.get(i);
-      // 64M
-      if (processor.memoryUsage() <= TSFileConfig.groupSizeInByte / 2) {
-        continue;
-      }
-      processor.writeLock();
-      try {
-        boolean isMerge = processor.flush().isHasOverflowFlushTask();
-        if (isMerge) {
-          processor.submitToMerge();
-        }
-      } finally {
-        processor.writeUnlock();
-      }
-    }
-  }
-
-  @Override
-  public void start() {
-    // do no thing
-  }
-
-  @Override
-  public void stop() {
-    try {
-      closeAll();
-    } catch (FileNodeManagerException e) {
-      LOGGER.error("Failed to close file node manager because .", e);
-    }
-  }
-
-  @Override
-  public ServiceType getID() {
-    return ServiceType.FILE_NODE_SERVICE;
-  }
-
-  /**
-   * get restore file path.
-   */
-  public String getRestoreFilePath(String processorName) {
-    FileNodeProcessor fileNodeProcessor = processorMap.get(processorName);
-    if (fileNodeProcessor != null) {
-      return fileNodeProcessor.getFileNodeRestoreFilePath();
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * recover filenode.
-   */
-  public void recoverFileNode(String filenodeName)
-      throws FileNodeManagerException {
-    FileNodeProcessor fileNodeProcessor = getProcessor(filenodeName, true);
-    LOGGER.info("Recover the filenode processor, the filenode is {}, the status is {}",
-        filenodeName, fileNodeProcessor.getFileNodeProcessorStatus());
-    try {
-      fileNodeProcessor.fileNodeRecovery();
-    } catch (FileNodeProcessorException e) {
-      throw new FileNodeManagerException(e);
-    } finally {
-      fileNodeProcessor.writeUnlock();
-    }
-  }
-
-  private enum FileNodeManagerStatus {
-    NONE, MERGE, CLOSE
-  }
-
-  private static class FileNodeManagerHolder {
-
-    private FileNodeManagerHolder() {
-    }
-
-    private static final FileNodeManager INSTANCE = new FileNodeManager(
-        TsFileDBConf.getFileNodeDir());
-  }
-
-}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java
deleted file mode 100644
index 786d4f3..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java
+++ /dev/null
@@ -1,2066 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-import static java.time.ZonedDateTime.ofInstant;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.time.Instant;
-import java.time.ZoneId;
-import java.time.ZonedDateTime;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.directories.Directories;
-import org.apache.iotdb.db.engine.Processor;
-import org.apache.iotdb.db.engine.bufferwrite.Action;
-import org.apache.iotdb.db.engine.bufferwrite.ActionException;
-import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor;
-import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants;
-import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.engine.overflow.io.OverflowProcessor;
-import org.apache.iotdb.db.engine.pool.MergeManager;
-import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource;
-import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
-import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile;
-import org.apache.iotdb.db.engine.version.SimpleFileVersionController;
-import org.apache.iotdb.db.engine.version.VersionController;
-import org.apache.iotdb.db.exception.BufferWriteProcessorException;
-import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
-import org.apache.iotdb.db.exception.ErrorDebugException;
-import org.apache.iotdb.db.exception.FileNodeProcessorException;
-import org.apache.iotdb.db.exception.OverflowProcessorException;
-import org.apache.iotdb.db.exception.PathErrorException;
-import org.apache.iotdb.db.exception.ProcessorException;
-import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.monitor.IStatistic;
-import org.apache.iotdb.db.monitor.MonitorConstants;
-import org.apache.iotdb.db.monitor.StatMonitor;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.apache.iotdb.db.utils.QueryUtils;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
-import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
-import org.apache.iotdb.tsfile.read.filter.TimeFilter;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
-import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.chunk.ChunkBuffer;
-import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
-import org.apache.iotdb.tsfile.write.schema.FileSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FileNodeProcessor extends Processor implements IStatistic {
-
-  private static final String WARN_NO_SUCH_OVERFLOWED_FILE = "Can not find any tsfile which"
-      + " will be overflowed in the filenode processor {}, ";
-  private static final String RESTORE_FILE_SUFFIX = ".restore";
-  private static final Logger LOGGER = LoggerFactory.getLogger(FileNodeProcessor.class);
-  private static final IoTDBConfig TsFileDBConf = IoTDBDescriptor.getInstance().getConfig();
-  private static final MManager mManager = MManager.getInstance();
-  private static final Directories directories = Directories.getInstance();
-  private final String statStorageDeltaName;
-  private final HashMap<String, AtomicLong> statParamsHashMap = new HashMap<>();
-  /**
-   * Used to keep the oldest timestamp for each deviceId. The key is deviceId.
-   */
-  private volatile boolean isOverflowed;
-  private Map<String, Long> lastUpdateTimeMap;
-  private Map<String, Long> flushLastUpdateTimeMap;
-  private Map<String, List<TsFileResource>> invertedIndexOfFiles;
-  private TsFileResource emptyTsFileResource;
-  private TsFileResource currentTsFileResource;
-  private List<TsFileResource> newFileNodes;
-  private FileNodeProcessorStatus isMerging;
-
-  /**
-   * this is used when work->merge operation
-   */
-  private int numOfMergeFile;
-  private FileNodeProcessorStore fileNodeProcessorStore;
-  private String fileNodeRestoreFilePath;
-  private final Object fileNodeRestoreLock = new Object();
-
-  /**
-   * last merge time
-   */
-  private long lastMergeTime = -1;
-  private BufferWriteProcessor bufferWriteProcessor = null;
-  private OverflowProcessor overflowProcessor = null;
-  private Set<Integer> oldMultiPassTokenSet = null;
-  private Set<Integer> newMultiPassTokenSet = new HashSet<>();
-
-  /**
-   * Represent the number of old queries that have not ended.
-   * This parameter only decreases but not increase.
-   */
-  private CountDownLatch oldMultiPassCount = null;
-
-  /**
-   * Represent the number of new queries that have not ended.
-   */
-  private AtomicInteger newMultiPassCount = new AtomicInteger(0);
-  /**
-   * system recovery
-   */
-  private boolean shouldRecovery = false;
-  /**
-   * statistic monitor parameters
-   */
-  private Map<String, Action> parameters;
-  private FileSchema fileSchema;
-  private Action flushFileNodeProcessorAction = () -> {
-    synchronized (fileNodeProcessorStore) {
-      try {
-        writeStoreToDisk(fileNodeProcessorStore);
-      } catch (FileNodeProcessorException e) {
-        throw new ActionException(e);
-      }
-    }
-  };
-  private Action bufferwriteFlushAction = () -> {
-    // update the lastUpdateTime Notice: Thread safe
-    synchronized (fileNodeProcessorStore) {
-      // deep copy
-      Map<String, Long> tempLastUpdateMap = new HashMap<>(lastUpdateTimeMap);
-      // update flushLastUpdateTimeMap
-      for (Entry<String, Long> entry : lastUpdateTimeMap.entrySet()) {
-        flushLastUpdateTimeMap.put(entry.getKey(), entry.getValue() + 1);
-      }
-      fileNodeProcessorStore.setLastUpdateTimeMap(tempLastUpdateMap);
-    }
-  };
-
-  private Action bufferwriteCloseAction = new Action() {
-
-    @Override
-    public void act() {
-      synchronized (fileNodeProcessorStore) {
-        fileNodeProcessorStore.setLastUpdateTimeMap(lastUpdateTimeMap);
-        addLastTimeToIntervalFile();
-        fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-      }
-    }
-
-    private void addLastTimeToIntervalFile() {
-
-      if (!newFileNodes.isEmpty()) {
-        // end time with one start time
-        Map<String, Long> endTimeMap = new HashMap<>();
-        for (Entry<String, Long> startTime : currentTsFileResource.getStartTimeMap().entrySet()) {
-          String deviceId = startTime.getKey();
-          endTimeMap.put(deviceId, lastUpdateTimeMap.get(deviceId));
-        }
-        currentTsFileResource.setEndTimeMap(endTimeMap);
-      }
-    }
-  };
-  private Action overflowFlushAction = () -> {
-
-    // update the new TsFileResource List and emptyIntervalFile.
-    // Notice: thread safe
-    synchronized (fileNodeProcessorStore) {
-      fileNodeProcessorStore.setOverflowed(isOverflowed);
-      fileNodeProcessorStore.setEmptyTsFileResource(emptyTsFileResource);
-      fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-    }
-  };
-  // Token for query which used to
-  private int multiPassLockToken = 0;
-  private VersionController versionController;
-  private ReentrantLock mergeDeleteLock = new ReentrantLock();
-
-  /**
-   * This is the modification file of the result of the current merge.
-   */
-  private ModificationFile mergingModification;
-
-  private TsFileIOWriter mergeFileWriter = null;
-  private String mergeOutputPath = null;
-  private String mergeBaseDir = null;
-  private String mergeFileName = null;
-  private boolean mergeIsChunkGroupHasData = false;
-  private long mergeStartPos;
-
-  /**
-   * constructor of FileNodeProcessor.
-   */
-  FileNodeProcessor(String fileNodeDirPath, String processorName)
-      throws FileNodeProcessorException {
-    super(processorName);
-    for (MonitorConstants.FileNodeProcessorStatConstants statConstant :
-        MonitorConstants.FileNodeProcessorStatConstants.values()) {
-      statParamsHashMap.put(statConstant.name(), new AtomicLong(0));
-    }
-    statStorageDeltaName =
-        MonitorConstants.STAT_STORAGE_GROUP_PREFIX + MonitorConstants.MONITOR_PATH_SEPARATOR
-            + MonitorConstants.FILE_NODE_PATH + MonitorConstants.MONITOR_PATH_SEPARATOR
-            + processorName.replaceAll("\\.", "_");
-
-    this.parameters = new HashMap<>();
-    String dirPath = fileNodeDirPath;
-    if (dirPath.length() > 0
-        && dirPath.charAt(dirPath.length() - 1) != File.separatorChar) {
-      dirPath = dirPath + File.separatorChar;
-    }
-
-    File restoreFolder = new File(dirPath + processorName);
-    if (!restoreFolder.exists()) {
-      restoreFolder.mkdirs();
-      LOGGER.info(
-          "The restore directory of the filenode processor {} doesn't exist. Create new " +
-              "directory {}",
-          getProcessorName(), restoreFolder.getAbsolutePath());
-    }
-    fileNodeRestoreFilePath = new File(restoreFolder, processorName + RESTORE_FILE_SUFFIX)
-        .getPath();
-    try {
-      fileNodeProcessorStore = readStoreFromDisk();
-    } catch (FileNodeProcessorException e) {
-      LOGGER.error(
-          "The fileNode processor {} encountered an error when recoverying restore " +
-              "information.", processorName);
-      throw new FileNodeProcessorException(e);
-    }
-    // TODO deep clone the lastupdate time
-    lastUpdateTimeMap = fileNodeProcessorStore.getLastUpdateTimeMap();
-    emptyTsFileResource = fileNodeProcessorStore.getEmptyTsFileResource();
-    newFileNodes = fileNodeProcessorStore.getNewFileNodes();
-    isMerging = fileNodeProcessorStore.getFileNodeProcessorStatus();
-    numOfMergeFile = fileNodeProcessorStore.getNumOfMergeFile();
-    invertedIndexOfFiles = new HashMap<>();
-    // deep clone
-    flushLastUpdateTimeMap = new HashMap<>();
-    for (Entry<String, Long> entry : lastUpdateTimeMap.entrySet()) {
-      flushLastUpdateTimeMap.put(entry.getKey(), entry.getValue() + 1);
-    }
-    // construct the fileschema
-    try {
-      this.fileSchema = constructFileSchema(processorName);
-    } catch (WriteProcessException e) {
-      throw new FileNodeProcessorException(e);
-    }
-    // status is not NONE, or the last intervalFile is not closed
-    if (isMerging != FileNodeProcessorStatus.NONE
-        || (!newFileNodes.isEmpty() && !newFileNodes.get(newFileNodes.size() - 1).isClosed())) {
-      shouldRecovery = true;
-    } else {
-      // add file into the index of file
-      addAllFileIntoIndex(newFileNodes);
-    }
-    // RegistStatService
-    if (TsFileDBConf.isEnableStatMonitor()) {
-      StatMonitor statMonitor = StatMonitor.getInstance();
-      registerStatMetadata();
-      statMonitor.registerStatistics(statStorageDeltaName, this);
-    }
-    try {
-      versionController = new SimpleFileVersionController(restoreFolder.getPath());
-    } catch (IOException e) {
-      throw new FileNodeProcessorException(e);
-    }
-  }
-
-  @Override
-  public Map<String, AtomicLong> getStatParamsHashMap() {
-    return statParamsHashMap;
-  }
-
-  @Override
-  public void registerStatMetadata() {
-    Map<String, String> hashMap = new HashMap<>();
-    for (MonitorConstants.FileNodeProcessorStatConstants statConstant :
-        MonitorConstants.FileNodeProcessorStatConstants.values()) {
-      hashMap
-          .put(statStorageDeltaName + MonitorConstants.MONITOR_PATH_SEPARATOR + statConstant.name(),
-              MonitorConstants.DATA_TYPE_INT64);
-    }
-    StatMonitor.getInstance().registerStatStorageGroup(hashMap);
-  }
-
-  @Override
-  public List<String> getAllPathForStatistic() {
-    List<String> list = new ArrayList<>();
-    for (MonitorConstants.FileNodeProcessorStatConstants statConstant :
-        MonitorConstants.FileNodeProcessorStatConstants.values()) {
-      list.add(
-          statStorageDeltaName + MonitorConstants.MONITOR_PATH_SEPARATOR + statConstant.name());
-    }
-    return list;
-  }
-
-  @Override
-  public Map<String, TSRecord> getAllStatisticsValue() {
-    Long curTime = System.currentTimeMillis();
-    HashMap<String, TSRecord> tsRecordHashMap = new HashMap<>();
-    TSRecord tsRecord = new TSRecord(curTime, statStorageDeltaName);
-
-    Map<String, AtomicLong> hashMap = getStatParamsHashMap();
-    tsRecord.dataPointList = new ArrayList<>();
-    for (Map.Entry<String, AtomicLong> entry : hashMap.entrySet()) {
-      tsRecord.dataPointList.add(new LongDataPoint(entry.getKey(), entry.getValue().get()));
-    }
-
-    tsRecordHashMap.put(statStorageDeltaName, tsRecord);
-    return tsRecordHashMap;
-  }
-
-  /**
-   * add interval FileNode.
-   */
-  void addIntervalFileNode(File file) throws ActionException, IOException {
-
-    TsFileResource tsFileResource = new TsFileResource(file, false);
-    this.currentTsFileResource = tsFileResource;
-    newFileNodes.add(tsFileResource);
-    fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-    flushFileNodeProcessorAction.act();
-  }
-
-  /**
-   * set interval filenode start time.
-   *
-   * @param deviceId device ID
-   */
-  void setIntervalFileNodeStartTime(String deviceId) {
-    if (currentTsFileResource.getStartTime(deviceId) == -1) {
-      currentTsFileResource.setStartTime(deviceId, flushLastUpdateTimeMap.get(deviceId));
-      if (!invertedIndexOfFiles.containsKey(deviceId)) {
-        invertedIndexOfFiles.put(deviceId, new ArrayList<>());
-      }
-      invertedIndexOfFiles.get(deviceId).add(currentTsFileResource);
-    }
-  }
-
-  void setIntervalFileNodeStartTime(String deviceId, long time) {
-    if (time != -1) {
-      currentTsFileResource.setStartTime(deviceId, time);
-    } else {
-      currentTsFileResource.removeTime(deviceId);
-      invertedIndexOfFiles.get(deviceId).remove(currentTsFileResource);
-    }
-  }
-
-  long getIntervalFileNodeStartTime(String deviceId) {
-    return currentTsFileResource.getStartTime(deviceId);
-  }
-
-  private void addAllFileIntoIndex(List<TsFileResource> fileList) {
-    // clear map
-    invertedIndexOfFiles.clear();
-    // add all file to index
-    for (TsFileResource fileNode : fileList) {
-      if (fileNode.getStartTimeMap().isEmpty()) {
-        continue;
-      }
-      for (String deviceId : fileNode.getStartTimeMap().keySet()) {
-        if (!invertedIndexOfFiles.containsKey(deviceId)) {
-          invertedIndexOfFiles.put(deviceId, new ArrayList<>());
-        }
-        invertedIndexOfFiles.get(deviceId).add(fileNode);
-      }
-    }
-  }
-
-  public boolean shouldRecovery() {
-    return shouldRecovery;
-  }
-
-  public boolean isOverflowed() {
-    return isOverflowed;
-  }
-
-  /**
-   * if overflow insert, update and delete write into this filenode processor, set
-   * <code>isOverflowed</code> to true.
-   */
-  public void setOverflowed(boolean isOverflowed) {
-    if (this.isOverflowed != isOverflowed) {
-      this.isOverflowed = isOverflowed;
-    }
-  }
-
-  public FileNodeProcessorStatus getFileNodeProcessorStatus() {
-    return isMerging;
-  }
-
-  /**
-   * execute filenode recovery.
-   */
-  public void fileNodeRecovery() throws FileNodeProcessorException, DiskSpaceInsufficientException {
-    // restore bufferwrite
-    if (!newFileNodes.isEmpty() && !newFileNodes.get(newFileNodes.size() - 1).isClosed()) {
-      //
-      // add the current file
-      //
-      currentTsFileResource = newFileNodes.get(newFileNodes.size() - 1);
-
-      // this bufferwrite file is not close by normal operation
-      String damagedFilePath = newFileNodes.get(newFileNodes.size() - 1).getFile().getAbsolutePath();
-      String[] fileNames = damagedFilePath.split("\\" + File.separator);
-      // all information to recovery the damaged file.
-      // contains file seriesPath, action parameters and processorName
-      parameters.put(FileNodeConstants.BUFFERWRITE_FLUSH_ACTION, bufferwriteFlushAction);
-      parameters.put(FileNodeConstants.BUFFERWRITE_CLOSE_ACTION, bufferwriteCloseAction);
-      parameters
-          .put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction);
-      String baseDir = directories
-          .getTsFileFolder(newFileNodes.get(newFileNodes.size() - 1).getBaseDirIndex());
-      if (LOGGER.isInfoEnabled()) {
-        LOGGER.info(
-            "The filenode processor {} will recovery the bufferwrite processor, "
-                + "the bufferwrite file is {}",
-            getProcessorName(), fileNames[fileNames.length - 1]);
-      }
-
-      try {
-        bufferWriteProcessor = new BufferWriteProcessor(baseDir, getProcessorName(),
-            fileNames[fileNames.length - 1], parameters, versionController, fileSchema);
-      } catch (BufferWriteProcessorException e) {
-        LOGGER.error(
-            "The filenode processor {} failed to recovery the bufferwrite processor, "
-                + "the last bufferwrite file is {}.",
-            getProcessorName(), fileNames[fileNames.length - 1]);
-        throw new FileNodeProcessorException(e);
-      }
-    }
-    // restore the overflow processor
-    LOGGER.info("The filenode processor {} will recovery the overflow processor.",
-        getProcessorName());
-    parameters.put(FileNodeConstants.OVERFLOW_FLUSH_ACTION, overflowFlushAction);
-    parameters.put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction);
-    try {
-      overflowProcessor = new OverflowProcessor(getProcessorName(), parameters, fileSchema,
-          versionController);
-    } catch (IOException e) {
-      LOGGER.error("The filenode processor {} failed to recovery the overflow processor.",
-          getProcessorName());
-      throw new FileNodeProcessorException(e);
-    }
-
-    shouldRecovery = false;
-
-    if (isMerging == FileNodeProcessorStatus.MERGING_WRITE) {
-      // re-merge all file
-      // if bufferwrite processor is not null, and close
-      LOGGER.info("The filenode processor {} is recovering, the filenode status is {}.",
-          getProcessorName(), isMerging);
-      merge();
-    } else if (isMerging == FileNodeProcessorStatus.WAITING) {
-      // unlock
-      LOGGER.info("The filenode processor {} is recovering, the filenode status is {}.",
-          getProcessorName(), isMerging);
-      //writeUnlock();
-      switchWaitingToWorking();
-    } else {
-      //writeUnlock();
-    }
-    // add file into index of file
-    addAllFileIntoIndex(newFileNodes);
-  }
-
-  /**
-   * get buffer write processor by processor name and insert time.
-   */
-  public BufferWriteProcessor getBufferWriteProcessor(String processorName, long insertTime)
-      throws FileNodeProcessorException, DiskSpaceInsufficientException {
-    if (bufferWriteProcessor == null) {
-      Map<String, Action> params = new HashMap<>();
-      params.put(FileNodeConstants.BUFFERWRITE_FLUSH_ACTION, bufferwriteFlushAction);
-      params.put(FileNodeConstants.BUFFERWRITE_CLOSE_ACTION, bufferwriteCloseAction);
-      params
-          .put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction);
-      String baseDir = directories.getNextFolderForTsfile();
-      LOGGER.info("Allocate folder {} for the new bufferwrite processor.", baseDir);
-      // construct processor or restore
-      try {
-        bufferWriteProcessor = new BufferWriteProcessor(baseDir, processorName,
-            insertTime + FileNodeConstants.BUFFERWRITE_FILE_SEPARATOR
-                + System.currentTimeMillis(),
-            params, versionController, fileSchema);
-      } catch (BufferWriteProcessorException e) {
-        throw new FileNodeProcessorException(String
-            .format("The filenode processor %s failed to get the bufferwrite processor.",
-                processorName), e);
-      }
-    } else if (bufferWriteProcessor.isClosed()) {
-      try {
-        bufferWriteProcessor.reopen(insertTime + FileNodeConstants.BUFFERWRITE_FILE_SEPARATOR
-            + System.currentTimeMillis());
-      } catch (BufferWriteProcessorException e) {
-        throw new FileNodeProcessorException("Cannot reopen BufferWriteProcessor", e);
-      }
-    }
-    return bufferWriteProcessor;
-  }
-
-  /**
-   * get overflow processor by processor name.
-   */
-  public OverflowProcessor getOverflowProcessor(String processorName) throws IOException {
-    if (overflowProcessor == null) {
-      Map<String, Action> params = new HashMap<>();
-      // construct processor or restore
-      params.put(FileNodeConstants.OVERFLOW_FLUSH_ACTION, overflowFlushAction);
-      params
-          .put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction);
-      overflowProcessor = new OverflowProcessor(processorName, params, fileSchema,
-          versionController);
-    } else if (overflowProcessor.isClosed()) {
-      overflowProcessor.reopen();
-    }
-    return overflowProcessor;
-  }
-
-  /**
-   * get overflow processor.
-   */
-  public OverflowProcessor getOverflowProcessor() {
-    if (overflowProcessor == null || overflowProcessor.isClosed()) {
-      LOGGER.error("The overflow processor is null when getting the overflowProcessor");
-    }
-    return overflowProcessor;
-  }
-
-  public boolean hasOverflowProcessor() {
-    return overflowProcessor != null && !overflowProcessor.isClosed();
-  }
-
-  public void setBufferwriteProcessroToClosed() {
-
-    bufferWriteProcessor = null;
-  }
-
-  public boolean hasBufferwriteProcessor() {
-
-    return bufferWriteProcessor != null;
-  }
-
-  /**
-   * set last update time.
-   */
-  public void setLastUpdateTime(String deviceId, long timestamp) {
-    if (!lastUpdateTimeMap.containsKey(deviceId) || lastUpdateTimeMap.get(deviceId) < timestamp) {
-      lastUpdateTimeMap.put(deviceId, timestamp);
-    }
-    if (timestamp == -1) {
-      lastUpdateTimeMap.remove(deviceId);
-    }
-  }
-
-  /**
-   * get last update time.
-   */
-  public long getLastUpdateTime(String deviceId) {
-
-    if (lastUpdateTimeMap.containsKey(deviceId)) {
-      return lastUpdateTimeMap.get(deviceId);
-    } else {
-      return -1;
-    }
-  }
-
-  /**
-   * get flush last update time.
-   */
-  public long getFlushLastUpdateTime(String deviceId) {
-    if (!flushLastUpdateTimeMap.containsKey(deviceId)) {
-      flushLastUpdateTimeMap.put(deviceId, 0L);
-    }
-    return flushLastUpdateTimeMap.get(deviceId);
-  }
-
-  public Map<String, Long> getLastUpdateTimeMap() {
-    return lastUpdateTimeMap;
-  }
-
-  /**
-   * For insert overflow.
-   */
-  public void changeTypeToChanged(String deviceId, long timestamp) {
-    if (!invertedIndexOfFiles.containsKey(deviceId)) {
-      LOGGER.warn(
-          WARN_NO_SUCH_OVERFLOWED_FILE
-              + "the data is [device:{},time:{}]",
-          getProcessorName(), deviceId, timestamp);
-      emptyTsFileResource.setStartTime(deviceId, 0L);
-      emptyTsFileResource.setEndTime(deviceId, getLastUpdateTime(deviceId));
-      emptyTsFileResource.changeTypeToChanged(isMerging);
-    } else {
-      List<TsFileResource> temp = invertedIndexOfFiles.get(deviceId);
-      int index = searchIndexNodeByTimestamp(deviceId, timestamp, temp);
-      changeTypeToChanged(temp.get(index), deviceId);
-    }
-  }
-
-  private void changeTypeToChanged(TsFileResource fileNode, String deviceId) {
-    fileNode.changeTypeToChanged(isMerging);
-    if (isMerging == FileNodeProcessorStatus.MERGING_WRITE) {
-      fileNode.addMergeChanged(deviceId);
-    }
-  }
-
-  /**
-   * For update overflow.
-   */
-  public void changeTypeToChanged(String deviceId, long startTime, long endTime) {
-    if (!invertedIndexOfFiles.containsKey(deviceId)) {
-      LOGGER.warn(
-          WARN_NO_SUCH_OVERFLOWED_FILE
-              + "the data is [device:{}, start time:{}, end time:{}]",
-          getProcessorName(), deviceId, startTime, endTime);
-      emptyTsFileResource.setStartTime(deviceId, 0L);
-      emptyTsFileResource.setEndTime(deviceId, getLastUpdateTime(deviceId));
-      emptyTsFileResource.changeTypeToChanged(isMerging);
-    } else {
-      List<TsFileResource> temp = invertedIndexOfFiles.get(deviceId);
-      int left = searchIndexNodeByTimestamp(deviceId, startTime, temp);
-      int right = searchIndexNodeByTimestamp(deviceId, endTime, temp);
-      for (int i = left; i <= right; i++) {
-        changeTypeToChanged(temp.get(i), deviceId);
-      }
-    }
-  }
-
-  /**
-   * For delete overflow.
-   */
-  public void changeTypeToChangedForDelete(String deviceId, long timestamp) {
-    if (!invertedIndexOfFiles.containsKey(deviceId)) {
-      LOGGER.warn(
-          WARN_NO_SUCH_OVERFLOWED_FILE
-              + "the data is [device:{}, delete time:{}]",
-          getProcessorName(), deviceId, timestamp);
-      emptyTsFileResource.setStartTime(deviceId, 0L);
-      emptyTsFileResource.setEndTime(deviceId, getLastUpdateTime(deviceId));
-      emptyTsFileResource.changeTypeToChanged(isMerging);
-    } else {
-      List<TsFileResource> temp = invertedIndexOfFiles.get(deviceId);
-      int index = searchIndexNodeByTimestamp(deviceId, timestamp, temp);
-      for (int i = 0; i <= index; i++) {
-        temp.get(i).changeTypeToChanged(isMerging);
-        if (isMerging == FileNodeProcessorStatus.MERGING_WRITE) {
-          temp.get(i).addMergeChanged(deviceId);
-        }
-      }
-    }
-  }
-
-  /**
-   * Search the index of the interval by the timestamp.
-   *
-   * @return index of interval
-   */
-  private int searchIndexNodeByTimestamp(String deviceId, long timestamp,
-      List<TsFileResource> fileList) {
-    int index = 1;
-    while (index < fileList.size()) {
-      if (timestamp < fileList.get(index).getStartTime(deviceId)) {
-        break;
-      } else {
-        index++;
-      }
-    }
-    return index - 1;
-  }
-
-  /**
-   * add multiple pass lock.
-   */
-  public int addMultiPassCount() {
-    LOGGER.debug("Add MultiPassCount: read lock newMultiPassCount.");
-    newMultiPassCount.incrementAndGet();
-    while (newMultiPassTokenSet.contains(multiPassLockToken)) {
-      multiPassLockToken++;
-    }
-    newMultiPassTokenSet.add(multiPassLockToken);
-    LOGGER.debug("Add multi token:{}, nsPath:{}.", multiPassLockToken, getProcessorName());
-    return multiPassLockToken;
-  }
-
-  /**
-   * decrease multiple pass count. TODO: use the return value or remove it.
-   */
-  public boolean decreaseMultiPassCount(int token) throws FileNodeProcessorException {
-    if (newMultiPassTokenSet.contains(token)) {
-      int newMultiPassCountValue = newMultiPassCount.decrementAndGet();
-      if (newMultiPassCountValue < 0) {
-        throw new FileNodeProcessorException(String
-            .format("Remove MultiPassCount error, newMultiPassCount:%d", newMultiPassCountValue));
-      }
-      newMultiPassTokenSet.remove(token);
-      LOGGER.debug("Remove multi token:{}, nspath:{}, new set:{}, count:{}", token,
-          getProcessorName(),
-          newMultiPassTokenSet, newMultiPassCount);
-      return true;
-    } else if (oldMultiPassTokenSet != null && oldMultiPassTokenSet.contains(token)) {
-      // remove token first, then unlock
-      oldMultiPassTokenSet.remove(token);
-      oldMultiPassCount.countDown();
-      long oldMultiPassCountValue = oldMultiPassCount.getCount();
-      if (oldMultiPassCountValue < 0) {
-        throw new FileNodeProcessorException(String
-            .format("Remove MultiPassCount error, oldMultiPassCount:%d", oldMultiPassCountValue));
-      }
-      LOGGER.debug("Remove multi token:{}, old set:{}, count:{}", token, oldMultiPassTokenSet,
-          oldMultiPassCount.getCount());
-      return true;
-    } else {
-      LOGGER.error("remove token error:{},new set:{}, old set:{}", token, newMultiPassTokenSet,
-          oldMultiPassTokenSet);
-      // should add throw exception
-      return false;
-    }
-  }
-
-  /**
-   * query data.
-   */
-  public <T extends Comparable<T>> QueryDataSource query(String deviceId, String measurementId,
-      QueryContext context) throws FileNodeProcessorException {
-    // query overflow data
-    MeasurementSchema mSchema;
-    TSDataType dataType;
-
-    //mSchema = mManager.getSchemaForOnePath(deviceId + "." + measurementId);
-    mSchema = fileSchema.getMeasurementSchema(measurementId);
-    dataType = mSchema.getType();
-
-    OverflowSeriesDataSource overflowSeriesDataSource;
-    try {
-      overflowSeriesDataSource = overflowProcessor.query(deviceId, measurementId, dataType,
-          mSchema.getProps(), context);
-    } catch (IOException e) {
-      throw new FileNodeProcessorException(e);
-    }
-    // tsfile dataØØ
-    List<TsFileResource> bufferwriteDataInFiles = new ArrayList<>();
-    for (TsFileResource tsFileResource : newFileNodes) {
-      // add the same tsFileResource, but not the same reference
-      if (tsFileResource.isClosed()) {
-        bufferwriteDataInFiles.add(tsFileResource.backUp());
-      }
-    }
-    Pair<ReadOnlyMemChunk, List<ChunkMetaData>> bufferwritedata = new Pair<>(null, null);
-    // bufferwrite data
-    UnsealedTsFile unsealedTsFile = null;
-
-    if (!newFileNodes.isEmpty() && !newFileNodes.get(newFileNodes.size() - 1).isClosed()
-        && !newFileNodes.get(newFileNodes.size() - 1).getStartTimeMap().isEmpty()) {
-      unsealedTsFile = new UnsealedTsFile();
-      unsealedTsFile.setFilePath(newFileNodes.get(newFileNodes.size() - 1).getFile().getAbsolutePath());
-      if (bufferWriteProcessor == null) {
-        throw new FileNodeProcessorException(String.format(
-            "The last of tsfile %s in filenode processor %s is not closed, "
-                + "but the bufferwrite processor is null.",
-            newFileNodes.get(newFileNodes.size() - 1).getFile().getAbsolutePath(), getProcessorName()));
-      }
-      try {
-        bufferwritedata = bufferWriteProcessor
-            .queryBufferWriteData(deviceId, measurementId, dataType, mSchema.getProps());
-      } catch (BufferWriteProcessorException e) {
-        throw new FileNodeProcessorException(e);
-      }
-
-      try {
-        List<Modification> pathModifications = context.getPathModifications(
-            currentTsFileResource.getModFile(), deviceId
-                + IoTDBConstant.PATH_SEPARATOR + measurementId
-        );
-        if (!pathModifications.isEmpty()) {
-          QueryUtils.modifyChunkMetaData(bufferwritedata.right, pathModifications);
-        }
-      } catch (IOException e) {
-        throw new FileNodeProcessorException(e);
-      }
-
-      unsealedTsFile.setTimeSeriesChunkMetaDatas(bufferwritedata.right);
-    }
-    GlobalSortedSeriesDataSource globalSortedSeriesDataSource = new GlobalSortedSeriesDataSource(
-        new Path(deviceId + "." + measurementId), bufferwriteDataInFiles, unsealedTsFile,
-        bufferwritedata.left);
-    return new QueryDataSource(globalSortedSeriesDataSource, overflowSeriesDataSource);
-
-  }
-
-  /**
-   * append one specified tsfile to this filenode processor.
-   *
-   * @param appendFile the appended tsfile information
-   * @param appendFilePath the seriesPath of appended file
-   */
-  public void appendFile(TsFileResource appendFile, String appendFilePath)
-      throws FileNodeProcessorException {
-    try {
-      if (!appendFile.getFile().getParentFile().exists()) {
-        appendFile.getFile().getParentFile().mkdirs();
-      }
-      // move file
-      File originFile = new File(appendFilePath);
-      File targetFile = appendFile.getFile();
-      if (!originFile.exists()) {
-        throw new FileNodeProcessorException(
-            String.format("The appended file %s does not exist.", appendFilePath));
-      }
-      if (targetFile.exists()) {
-        throw new FileNodeProcessorException(
-            String.format("The appended target file %s already exists.",
-                appendFile.getFile().getAbsolutePath()));
-      }
-      if (!originFile.renameTo(targetFile)) {
-        LOGGER.warn("File renaming failed when appending new file. Origin: {}, Target: {}",
-            originFile.getPath(), targetFile.getPath());
-      }
-      // append the new tsfile
-      this.newFileNodes.add(appendFile);
-      // update the lastUpdateTime
-      for (Entry<String, Long> entry : appendFile.getEndTimeMap().entrySet()) {
-        lastUpdateTimeMap.put(entry.getKey(), entry.getValue());
-      }
-      bufferwriteFlushAction.act();
-      fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-      // reconstruct the inverted index of the newFileNodes
-      flushFileNodeProcessorAction.act();
-      addAllFileIntoIndex(newFileNodes);
-    } catch (Exception e) {
-      LOGGER.error("Failed to append the tsfile {} to filenode processor {}.", appendFile,
-          getProcessorName());
-      throw new FileNodeProcessorException(e);
-    }
-  }
-
-  /**
-   * get overlap tsfiles which are conflict with the appendFile.
-   *
-   * @param appendFile the appended tsfile information
-   */
-  public List<String> getOverlapFiles(TsFileResource appendFile, String uuid)
-      throws FileNodeProcessorException {
-    List<String> overlapFiles = new ArrayList<>();
-    try {
-      for (TsFileResource tsFileResource : newFileNodes) {
-        getOverlapFiles(appendFile, tsFileResource, uuid, overlapFiles);
-      }
-    } catch (IOException e) {
-      LOGGER.error("Failed to get overlap tsfiles which conflict with the appendFile.");
-      throw new FileNodeProcessorException(e);
-    }
-    return overlapFiles;
-  }
-
-  private void getOverlapFiles(TsFileResource appendFile, TsFileResource tsFileResource,
-      String uuid, List<String> overlapFiles) throws IOException {
-    for (Entry<String, Long> entry : appendFile.getStartTimeMap().entrySet()) {
-      if (tsFileResource.getStartTimeMap().containsKey(entry.getKey()) &&
-          tsFileResource.getEndTime(entry.getKey()) >= entry.getValue()
-          && tsFileResource.getStartTime(entry.getKey()) <= appendFile
-          .getEndTime(entry.getKey())) {
-        String relativeFilePath =
-            Constans.SYNC_SERVER + File.separatorChar + uuid + File.separatorChar
-                + Constans.BACK_UP_DIRECTORY_NAME
-                + File.separatorChar + tsFileResource.getRelativePath();
-        File newFile = new File(
-            Directories.getInstance().getTsFileFolder(tsFileResource.getBaseDirIndex()),
-            relativeFilePath);
-        if (!newFile.getParentFile().exists()) {
-          newFile.getParentFile().mkdirs();
-        }
-        java.nio.file.Path link = FileSystems.getDefault().getPath(newFile.getPath());
-        java.nio.file.Path target = FileSystems.getDefault()
-            .getPath(tsFileResource.getFile().getAbsolutePath());
-        Files.createLink(link, target);
-        overlapFiles.add(newFile.getPath());
-        break;
-      }
-    }
-  }
-
-  /**
-   * add time series.
-   */
-  public void addTimeSeries(String measurementId, TSDataType dataType, TSEncoding encoding,
-      CompressionType compressor, Map<String, String> props) {
-    fileSchema.registerMeasurement(new MeasurementSchema(measurementId, dataType, encoding,
-        compressor, props));
-  }
-
-  /**
-   * submit the merge task to the <code>MergePool</code>.
-   *
-   * @return null -can't submit the merge task, because this filenode is not overflowed or it is
-   * merging now. Future - submit the merge task successfully.
-   */
-  Future submitToMerge() {
-    ZoneId zoneId = IoTDBDescriptor.getInstance().getConfig().getZoneID();
-    if (lastMergeTime > 0) {
-      long thisMergeTime = System.currentTimeMillis();
-      long mergeTimeInterval = thisMergeTime - lastMergeTime;
-      ZonedDateTime lastDateTime = ofInstant(Instant.ofEpochMilli(lastMergeTime),
-          zoneId);
-      ZonedDateTime thisDateTime = ofInstant(Instant.ofEpochMilli(thisMergeTime),
-          zoneId);
-      LOGGER.info(
-          "The filenode {} last merge time is {}, this merge time is {}, "
-              + "merge time interval is {}s",
-          getProcessorName(), lastDateTime, thisDateTime, mergeTimeInterval / 1000);
-    }
-    lastMergeTime = System.currentTimeMillis();
-
-    if (overflowProcessor != null && !overflowProcessor.isClosed()) {
-      if (overflowProcessor.getFileSize() < IoTDBDescriptor.getInstance()
-          .getConfig().getOverflowFileSizeThreshold()) {
-        if (LOGGER.isInfoEnabled()) {
-          LOGGER.info(
-              "Skip this merge taks submission, because the size{} of overflow processor {} "
-                  + "does not reaches the threshold {}.",
-              MemUtils.bytesCntToStr(overflowProcessor.getFileSize()), getProcessorName(),
-              MemUtils.bytesCntToStr(
-                  IoTDBDescriptor.getInstance().getConfig().getOverflowFileSizeThreshold()));
-        }
-        return null;
-      }
-    } else {
-      LOGGER.info(
-          "Skip this merge taks submission, because the filenode processor {} "
-              + "has no overflow processor.",
-          getProcessorName());
-      return null;
-    }
-    if (isOverflowed && isMerging == FileNodeProcessorStatus.NONE) {
-      Runnable mergeThread;
-      mergeThread = new MergeRunnale();
-      LOGGER.info("Submit the merge task, the merge filenode is {}", getProcessorName());
-      return MergeManager.getInstance().submit(mergeThread);
-    } else {
-      if (!isOverflowed) {
-        LOGGER.info(
-            "Skip this merge taks submission, because the filenode processor {} is not " +
-                "overflowed.",
-            getProcessorName());
-      } else {
-        LOGGER.warn(
-            "Skip this merge task submission, because last merge task is not over yet, "
-                + "the merge filenode processor is {}",
-            getProcessorName());
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Prepare for merge, close the bufferwrite and overflow.
-   */
-  private void prepareForMerge() {
-    try {
-      LOGGER.info("The filenode processor {} prepares for merge, closes the bufferwrite processor",
-          getProcessorName());
-      closeBufferWrite();
-      // try to get overflow processor
-      getOverflowProcessor(getProcessorName());
-      // must close the overflow processor
-      while (!getOverflowProcessor().canBeClosed()) {
-        waitForClosing();
-      }
-      LOGGER.info("The filenode processor {} prepares for merge, closes the overflow processor",
-          getProcessorName());
-      getOverflowProcessor().close();
-    } catch (FileNodeProcessorException | OverflowProcessorException | IOException e) {
-      LOGGER.error("The filenode processor {} prepares for merge error.", getProcessorName());
-      writeUnlock();
-      throw new ErrorDebugException(e);
-    }
-  }
-
-  private void waitForClosing() {
-    try {
-      LOGGER.info(
-          "The filenode processor {} prepares for merge, the overflow {} can't be closed, "
-              + "wait 100ms,",
-          getProcessorName(), getProcessorName());
-      TimeUnit.MICROSECONDS.sleep(100);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  /**
-   * Merge this storage group, merge the tsfile data with overflow data.
-   */
-  public void merge() throws FileNodeProcessorException, DiskSpaceInsufficientException {
-    // close bufferwrite and overflow, prepare for merge
-    LOGGER.info("The filenode processor {} begins to merge.", getProcessorName());
-    prepareForMerge();
-    // change status from overflowed to no overflowed
-    isOverflowed = false;
-    // change status from work to merge
-    isMerging = FileNodeProcessorStatus.MERGING_WRITE;
-    // check the empty file
-    Map<String, Long> startTimeMap = emptyTsFileResource.getStartTimeMap();
-    mergeCheckEmptyFile(startTimeMap);
-
-    for (TsFileResource tsFileResource : newFileNodes) {
-      if (tsFileResource.getOverflowChangeType() != OverflowChangeType.NO_CHANGE) {
-        tsFileResource.setOverflowChangeType(OverflowChangeType.CHANGED);
-      }
-    }
-
-    addAllFileIntoIndex(newFileNodes);
-    synchronized (fileNodeProcessorStore) {
-      fileNodeProcessorStore.setOverflowed(isOverflowed);
-      fileNodeProcessorStore.setFileNodeProcessorStatus(isMerging);
-      fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-      fileNodeProcessorStore.setEmptyTsFileResource(emptyTsFileResource);
-      // flush this filenode information
-      try {
-        writeStoreToDisk(fileNodeProcessorStore);
-      } catch (FileNodeProcessorException e) {
-        LOGGER.error("The filenode processor {} writes restore information error when merging.",
-            getProcessorName(), e);
-        writeUnlock();
-        throw new FileNodeProcessorException(e);
-      }
-    }
-    // add numOfMergeFile to control the number of the merge file
-    List<TsFileResource> backupIntervalFiles;
-
-    backupIntervalFiles = switchFileNodeToMerge();
-    //
-    // clear empty file
-    //
-    boolean needEmtpy = false;
-    if (emptyTsFileResource.getOverflowChangeType() != OverflowChangeType.NO_CHANGE) {
-      needEmtpy = true;
-    }
-    emptyTsFileResource.clear();
-    // attention
-    try {
-      if (overflowProcessor.isClosed()) {
-        overflowProcessor.reopen();
-      }
-      overflowProcessor.switchWorkToMerge();
-    } catch (IOException e) {
-      LOGGER.error("The filenode processor {} can't switch overflow processor from work to merge.",
-          getProcessorName(), e);
-      writeUnlock();
-      throw new FileNodeProcessorException(e);
-    }
-    LOGGER.info("The filenode processor {} switches from {} to {}.", getProcessorName(),
-        FileNodeProcessorStatus.NONE, FileNodeProcessorStatus.MERGING_WRITE);
-    writeUnlock();
-
-    // query tsfile data and overflow data, and merge them
-    int numOfMergeFiles = 0;
-    int allNeedMergeFiles = backupIntervalFiles.size();
-    for (TsFileResource backupIntervalFile : backupIntervalFiles) {
-      numOfMergeFiles++;
-      if (backupIntervalFile.getOverflowChangeType() == OverflowChangeType.CHANGED) {
-        // query data and merge
-        String filePathBeforeMerge = backupIntervalFile.getRelativePath();
-        try {
-          LOGGER.info(
-              "The filenode processor {} begins merging the {}/{} tsfile[{}] with "
-                  + "overflow file, the process is {}%",
-              getProcessorName(), numOfMergeFiles, allNeedMergeFiles, filePathBeforeMerge,
-              (int) (((numOfMergeFiles - 1) / (float) allNeedMergeFiles) * 100));
-          long startTime = System.currentTimeMillis();
-          String newFile = queryAndWriteDataForMerge(backupIntervalFile);
-          long endTime = System.currentTimeMillis();
-          long timeConsume = endTime - startTime;
-          ZoneId zoneId = IoTDBDescriptor.getInstance().getConfig().getZoneID();
-          LOGGER.info(
-              "The fileNode processor {} has merged the {}/{} tsfile[{}->{}] over, "
-                  + "start time of merge is {}, end time of merge is {}, "
-                  + "time consumption is {}ms,"
-                  + " the process is {}%",
-              getProcessorName(), numOfMergeFiles, allNeedMergeFiles, filePathBeforeMerge,
-              newFile, ofInstant(Instant.ofEpochMilli(startTime),
-                  zoneId), ofInstant(Instant.ofEpochMilli(endTime), zoneId), timeConsume,
-              numOfMergeFiles / (float) allNeedMergeFiles * 100);
-        } catch (IOException | PathErrorException e) {
-          LOGGER.error("Merge: query and write data error.", e);
-          throw new FileNodeProcessorException(e);
-        }
-      } else if (backupIntervalFile.getOverflowChangeType() == OverflowChangeType.MERGING_CHANGE) {
-        LOGGER.error("The overflowChangeType of backupIntervalFile must not be {}",
-            OverflowChangeType.MERGING_CHANGE);
-        // handle this error, throw one runtime exception
-        throw new FileNodeProcessorException(
-            "The overflowChangeType of backupIntervalFile must not be "
-                + OverflowChangeType.MERGING_CHANGE);
-      } else {
-        LOGGER.debug(
-            "The filenode processor {} is merging, the interval file {} doesn't "
-                + "need to be merged.",
-            getProcessorName(), backupIntervalFile.getRelativePath());
-      }
-    }
-
-    // change status from merge to wait
-    switchMergeToWaiting(backupIntervalFiles, needEmtpy);
-
-    // change status from wait to work
-    switchWaitingToWorking();
-  }
-
-  private void mergeCheckEmptyFile(Map<String, Long> startTimeMap) {
-    if (emptyTsFileResource.getOverflowChangeType() == OverflowChangeType.NO_CHANGE) {
-      return;
-    }
-    Iterator<Entry<String, Long>> iterator = emptyTsFileResource.getEndTimeMap().entrySet()
-        .iterator();
-    while (iterator.hasNext()) {
-      Entry<String, Long> entry = iterator.next();
-      String deviceId = entry.getKey();
-      if (invertedIndexOfFiles.containsKey(deviceId)) {
-        invertedIndexOfFiles.get(deviceId).get(0).setOverflowChangeType(OverflowChangeType.CHANGED);
-        startTimeMap.remove(deviceId);
-        iterator.remove();
-      }
-    }
-    if (emptyTsFileResource.checkEmpty()) {
-      emptyTsFileResource.clear();
-    } else {
-      if (!newFileNodes.isEmpty()) {
-        TsFileResource first = newFileNodes.get(0);
-        for (String deviceId : emptyTsFileResource.getStartTimeMap().keySet()) {
-          first.setStartTime(deviceId, emptyTsFileResource.getStartTime(deviceId));
-          first.setEndTime(deviceId, emptyTsFileResource.getEndTime(deviceId));
-          first.setOverflowChangeType(OverflowChangeType.CHANGED);
-        }
-        emptyTsFileResource.clear();
-      } else {
-        emptyTsFileResource.setOverflowChangeType(OverflowChangeType.CHANGED);
-      }
-    }
-  }
-
-  private List<TsFileResource> switchFileNodeToMerge() throws FileNodeProcessorException {
-    List<TsFileResource> result = new ArrayList<>();
-    if (emptyTsFileResource.getOverflowChangeType() != OverflowChangeType.NO_CHANGE) {
-      // add empty
-      result.add(emptyTsFileResource.backUp());
-      if (!newFileNodes.isEmpty()) {
-        throw new FileNodeProcessorException(
-            String.format("The status of empty file is %s, but the new file list is not empty",
-                emptyTsFileResource.getOverflowChangeType()));
-      }
-      return result;
-    }
-    if (newFileNodes.isEmpty()) {
-      LOGGER.error("No file was changed when merging, the filenode is {}", getProcessorName());
-      throw new FileNodeProcessorException(
-          "No file was changed when merging, the filenode is " + getProcessorName());
-    }
-    for (TsFileResource tsFileResource : newFileNodes) {
-      updateFileNode(tsFileResource, result);
-    }
-    return result;
-  }
-
-  private void updateFileNode(TsFileResource tsFileResource, List<TsFileResource> result) {
-    if (tsFileResource.getOverflowChangeType() == OverflowChangeType.NO_CHANGE) {
-      result.add(tsFileResource.backUp());
-    } else {
-      Map<String, Long> startTimeMap = new HashMap<>();
-      Map<String, Long> endTimeMap = new HashMap<>();
-      for (String deviceId : tsFileResource.getEndTimeMap().keySet()) {
-        List<TsFileResource> temp = invertedIndexOfFiles.get(deviceId);
-        int index = temp.indexOf(tsFileResource);
-        int size = temp.size();
-        // start time
-        if (index == 0) {
-          startTimeMap.put(deviceId, 0L);
-        } else {
-          startTimeMap.put(deviceId, tsFileResource.getStartTime(deviceId));
-        }
-        // end time
-        if (index < size - 1) {
-          endTimeMap.put(deviceId, temp.get(index + 1).getStartTime(deviceId) - 1);
-        } else {
-          endTimeMap.put(deviceId, tsFileResource.getEndTime(deviceId));
-        }
-      }
-      TsFileResource node = new TsFileResource(startTimeMap, endTimeMap,
-          tsFileResource.getOverflowChangeType(), tsFileResource.getFile());
-      result.add(node);
-    }
-  }
-
-  private void switchMergeToWaiting(List<TsFileResource> backupIntervalFiles, boolean needEmpty)
-      throws FileNodeProcessorException {
-    LOGGER.info("The status of filenode processor {} switches from {} to {}.", getProcessorName(),
-        FileNodeProcessorStatus.MERGING_WRITE, FileNodeProcessorStatus.WAITING);
-    writeLock();
-    try {
-      oldMultiPassTokenSet = newMultiPassTokenSet;
-      oldMultiPassCount = new CountDownLatch(newMultiPassCount.get());
-      newMultiPassTokenSet = new HashSet<>();
-      newMultiPassCount = new AtomicInteger(0);
-      List<TsFileResource> result = new ArrayList<>();
-      int beginIndex = 0;
-      if (needEmpty) {
-        TsFileResource empty = backupIntervalFiles.get(0);
-        if (!empty.checkEmpty()) {
-          updateEmpty(empty, result);
-          beginIndex++;
-        }
-      }
-      // reconstruct the file index
-      addAllFileIntoIndex(backupIntervalFiles);
-      // check the merge changed file
-      for (int i = beginIndex; i < backupIntervalFiles.size(); i++) {
-        TsFileResource newFile = newFileNodes.get(i - beginIndex);
-        TsFileResource temp = backupIntervalFiles.get(i);
-        if (newFile.getOverflowChangeType() == OverflowChangeType.MERGING_CHANGE) {
-          updateMergeChanged(newFile, temp);
-        }
-        if (!temp.checkEmpty()) {
-          result.add(temp);
-        }
-      }
-      // add new file when merge
-      for (int i = backupIntervalFiles.size() - beginIndex; i < newFileNodes.size(); i++) {
-        TsFileResource fileNode = newFileNodes.get(i);
-        if (fileNode.isClosed()) {
-          result.add(fileNode.backUp());
-        } else {
-          result.add(fileNode);
-        }
-      }
-
-      isMerging = FileNodeProcessorStatus.WAITING;
-      newFileNodes = result;
-      // reconstruct the index
-      addAllFileIntoIndex(newFileNodes);
-      // clear merge changed
-      for (TsFileResource fileNode : newFileNodes) {
-        fileNode.clearMergeChanged();
-      }
-
-      synchronized (fileNodeProcessorStore) {
-        fileNodeProcessorStore.setFileNodeProcessorStatus(isMerging);
-        fileNodeProcessorStore.setEmptyTsFileResource(emptyTsFileResource);
-        fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-        try {
-          writeStoreToDisk(fileNodeProcessorStore);
-        } catch (FileNodeProcessorException e) {
-          LOGGER.error(
-              "Merge: failed to write filenode information to revocery file, the filenode is " +
-                  "{}.",
-              getProcessorName(), e);
-          throw new FileNodeProcessorException(
-              "Merge: write filenode information to revocery file failed, the filenode is "
-                  + getProcessorName());
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  private void updateEmpty(TsFileResource empty, List<TsFileResource> result) {
-    for (String deviceId : empty.getStartTimeMap().keySet()) {
-      if (invertedIndexOfFiles.containsKey(deviceId)) {
-        TsFileResource temp = invertedIndexOfFiles.get(deviceId).get(0);
-        if (temp.getMergeChanged().contains(deviceId)) {
-          empty.setOverflowChangeType(OverflowChangeType.CHANGED);
-          break;
-        }
-      }
-    }
-    empty.clearMergeChanged();
-    result.add(empty.backUp());
-  }
-
-  private void updateMergeChanged(TsFileResource newFile, TsFileResource temp) {
-    for (String deviceId : newFile.getMergeChanged()) {
-      if (temp.getStartTimeMap().containsKey(deviceId)) {
-        temp.setOverflowChangeType(OverflowChangeType.CHANGED);
-      } else {
-        changeTypeToChanged(deviceId, newFile.getStartTime(deviceId),
-            newFile.getEndTime(deviceId));
-      }
-    }
-  }
-
-
-  private void switchWaitingToWorking()
-      throws FileNodeProcessorException {
-
-    LOGGER.info("The status of filenode processor {} switches from {} to {}.", getProcessorName(),
-        FileNodeProcessorStatus.WAITING, FileNodeProcessorStatus.NONE);
-
-    if (oldMultiPassCount != null) {
-      LOGGER.info("The old Multiple Pass Token set is {}, the old Multiple Pass Count is {}",
-          oldMultiPassTokenSet,
-          oldMultiPassCount);
-      try {
-        oldMultiPassCount.await();
-      } catch (InterruptedException e) {
-        LOGGER.info(
-            "The filenode processor {} encountered an error when it waits for all old queries over.",
-            getProcessorName());
-        throw new FileNodeProcessorException(e);
-      }
-    }
-
-    try {
-      writeLock();
-      try {
-        // delete the all files which are in the newFileNodes
-        // notice: the last restore file of the interval file
-
-        List<String> bufferwriteDirPathList = directories.getAllTsFileFolders();
-        List<File> bufferwriteDirList = new ArrayList<>();
-        collectBufferWriteDirs(bufferwriteDirPathList, bufferwriteDirList);
-
-        Set<String> bufferFiles = new HashSet<>();
-        collectBufferWriteFiles(bufferFiles);
-
-        // add the restore file, if the last file is not closed
-        if (!newFileNodes.isEmpty() && !newFileNodes.get(newFileNodes.size() - 1).isClosed()) {
-          String bufferFileRestorePath =
-              newFileNodes.get(newFileNodes.size() - 1).getFile().getAbsolutePath() + RESTORE_FILE_SUFFIX;
-          bufferFiles.add(bufferFileRestorePath);
-        }
-
-        deleteBufferWriteFiles(bufferwriteDirList, bufferFiles);
-
-        // merge switch
-        changeFileNodes();
-
-        // overflow switch from merge to work
-        overflowProcessor.switchMergeToWork();
-
-        // write status to file
-        isMerging = FileNodeProcessorStatus.NONE;
-        synchronized (fileNodeProcessorStore) {
-          fileNodeProcessorStore.setFileNodeProcessorStatus(isMerging);
-          fileNodeProcessorStore.setNewFileNodes(newFileNodes);
-          fileNodeProcessorStore.setEmptyTsFileResource(emptyTsFileResource);
-          writeStoreToDisk(fileNodeProcessorStore);
-        }
-      } catch (IOException e) {
-        LOGGER.info(
-            "The filenode processor {} encountered an error when its "
-                + "status switched from {} to {}.",
-            getProcessorName(), FileNodeProcessorStatus.NONE,
-            FileNodeProcessorStatus.MERGING_WRITE);
-        throw new FileNodeProcessorException(e);
-      } finally {
-        writeUnlock();
-      }
-    } finally {
-      oldMultiPassTokenSet = null;
-      oldMultiPassCount = null;
-    }
-
-  }
-
-  private void collectBufferWriteDirs(List<String> bufferwriteDirPathList,
-      List<File> bufferwriteDirList) {
-    for (String bufferwriteDirPath : bufferwriteDirPathList) {
-      if (bufferwriteDirPath.length() > 0
-          && bufferwriteDirPath.charAt(bufferwriteDirPath.length() - 1)
-          != File.separatorChar) {
-        bufferwriteDirPath = bufferwriteDirPath + File.separatorChar;
-      }
-      bufferwriteDirPath = bufferwriteDirPath + getProcessorName();
-      File bufferwriteDir = new File(bufferwriteDirPath);
-      bufferwriteDirList.add(bufferwriteDir);
-      if (!bufferwriteDir.exists()) {
-        bufferwriteDir.mkdirs();
-      }
-    }
-  }
-
-  private void collectBufferWriteFiles(Set<String> bufferFiles) {
-    for (TsFileResource bufferFileNode : newFileNodes) {
-      String bufferFilePath = bufferFileNode.getFile().getAbsolutePath();
-      if (bufferFilePath != null) {
-        bufferFiles.add(bufferFilePath);
-      }
-    }
-  }
-
-  private void deleteBufferWriteFiles(List<File> bufferwriteDirList, Set<String> bufferFiles)
-      throws IOException {
-    for (File bufferwriteDir : bufferwriteDirList) {
-      File[] files = bufferwriteDir.listFiles();
-      if (files == null) {
-        continue;
-      }
-      for (File file : files) {
-        if (!bufferFiles.contains(file.getPath())) {
-          FileReaderManager.getInstance().closeFileAndRemoveReader(file.getPath());
-          if (!file.delete()) {
-            LOGGER.warn("Cannot delete BufferWrite file {}", file.getPath());
-          }
-        }
-      }
-    }
-  }
-
-  private void changeFileNodes() {
-    for (TsFileResource fileNode : newFileNodes) {
-      if (fileNode.getOverflowChangeType() != OverflowChangeType.NO_CHANGE) {
-        fileNode.setOverflowChangeType(OverflowChangeType.CHANGED);
-      }
-    }
-  }
-
-  private String queryAndWriteDataForMerge(TsFileResource backupIntervalFile)
-      throws IOException, FileNodeProcessorException, PathErrorException, DiskSpaceInsufficientException {
-    Map<String, Long> startTimeMap = new HashMap<>();
-    Map<String, Long> endTimeMap = new HashMap<>();
-
-    mergeFileWriter = null;
-    mergeOutputPath = null;
-    mergeBaseDir = null;
-    mergeFileName = null;
-    // modifications are blocked before mergeModification is created to avoid
-    // losing some modification.
-    mergeDeleteLock.lock();
-    QueryContext context = new QueryContext();
-    try {
-      FileReaderManager.getInstance().increaseFileReaderReference(backupIntervalFile.getFilePath(),
-          true);
-      for (String deviceId : backupIntervalFile.getStartTimeMap().keySet()) {
-        // query one deviceId
-        List<Path> pathList = new ArrayList<>();
-        mergeIsChunkGroupHasData = false;
-        mergeStartPos = -1;
-        ChunkGroupFooter footer;
-        int numOfChunk = 0;
-        try {
-          List<String> pathStrings = mManager.getLeafNodePathInNextLevel(deviceId);
-          for (String string : pathStrings) {
-            pathList.add(new Path(string));
-          }
-        } catch (PathErrorException e) {
-          LOGGER.error("Can't get all the paths from MManager, the deviceId is {}", deviceId);
-          throw new FileNodeProcessorException(e);
-        }
-        if (pathList.isEmpty()) {
-          continue;
-        }
-        for (Path path : pathList) {
-          // query one measurement in the special deviceId
-          String measurementId = path.getMeasurement();
-          TSDataType dataType = mManager.getSeriesType(path.getFullPath());
-          OverflowSeriesDataSource overflowSeriesDataSource = overflowProcessor.queryMerge(deviceId,
-              measurementId, dataType, true, context);
-          Filter timeFilter = FilterFactory
-              .and(TimeFilter.gtEq(backupIntervalFile.getStartTime(deviceId)),
-                  TimeFilter.ltEq(backupIntervalFile.getEndTime(deviceId)));
-          SingleSeriesExpression seriesFilter = new SingleSeriesExpression(path, timeFilter);
-
-          for (OverflowInsertFile overflowInsertFile : overflowSeriesDataSource
-              .getOverflowInsertFileList()) {
-            FileReaderManager.getInstance()
-                .increaseFileReaderReference(overflowInsertFile.getFilePath(),
-                    false);
-          }
-
-          IReader seriesReader = SeriesReaderFactory.getInstance()
-              .createSeriesReaderForMerge(backupIntervalFile,
-                  overflowSeriesDataSource, seriesFilter, context);
-          numOfChunk += queryAndWriteSeries(seriesReader, path, seriesFilter, dataType,
-              startTimeMap, endTimeMap, overflowSeriesDataSource);
-        }
-        if (mergeIsChunkGroupHasData) {
-          // end the new rowGroupMetadata
-          long size = mergeFileWriter.getPos() - mergeStartPos;
-          footer = new ChunkGroupFooter(deviceId, size, numOfChunk);
-          mergeFileWriter.endChunkGroup(footer, 0);
-        }
-      }
-    } finally {
-      FileReaderManager.getInstance().decreaseFileReaderReference(backupIntervalFile.getFilePath(),
-          true);
-
-      if (mergeDeleteLock.isLocked()) {
-        mergeDeleteLock.unlock();
-      }
-    }
-
-    if (mergeFileWriter != null) {
-      mergeFileWriter.endFile(fileSchema);
-    }
-    backupIntervalFile.setFile(new File(mergeBaseDir + File.separator + mergeFileName));
-    backupIntervalFile.setOverflowChangeType(OverflowChangeType.NO_CHANGE);
-    backupIntervalFile.setStartTimeMap(startTimeMap);
-    backupIntervalFile.setEndTimeMap(endTimeMap);
-    backupIntervalFile.setModFile(mergingModification);
-    mergingModification = null;
-    return mergeFileName;
-  }
-
-  private int queryAndWriteSeries(IReader seriesReader, Path path,
-      SingleSeriesExpression seriesFilter, TSDataType dataType,
-      Map<String, Long> startTimeMap, Map<String, Long> endTimeMap,
-      OverflowSeriesDataSource overflowSeriesDataSource)
-      throws IOException, DiskSpaceInsufficientException {
-    int numOfChunk = 0;
-    try {
-      if (!seriesReader.hasNext()) {
-        LOGGER.debug(
-            "The time-series {} has no data with the filter {} in the filenode processor {}",
-            path, seriesFilter, getProcessorName());
-      } else {
-        numOfChunk++;
-        TimeValuePair timeValuePair = seriesReader.next();
-        if (mergeFileWriter == null) {
-          mergeBaseDir = directories.getNextFolderForTsfile();
-          mergeFileName = timeValuePair.getTimestamp()
-              + FileNodeConstants.BUFFERWRITE_FILE_SEPARATOR + System.currentTimeMillis();
-          mergeOutputPath = constructOutputFilePath(mergeBaseDir, getProcessorName(),
-              mergeFileName);
-          mergeFileName = getProcessorName() + File.separatorChar + mergeFileName;
-          mergeFileWriter = new TsFileIOWriter(new File(mergeOutputPath));
-          mergingModification = new ModificationFile(mergeOutputPath
-              + ModificationFile.FILE_SUFFIX);
-          mergeDeleteLock.unlock();
-        }
-        if (!mergeIsChunkGroupHasData) {
-          // start a new rowGroupMetadata
-          mergeIsChunkGroupHasData = true;
-          // the datasize and numOfChunk is fake
-          // the accurate datasize and numOfChunk will get after write all this device data.
-          mergeFileWriter.startFlushChunkGroup(path.getDevice());// TODO please check me.
-          mergeStartPos = mergeFileWriter.getPos();
-        }
-        // init the serieswWriteImpl
-        MeasurementSchema measurementSchema = fileSchema
-            .getMeasurementSchema(path.getMeasurement());
-        ChunkBuffer pageWriter = new ChunkBuffer(measurementSchema);
-        int pageSizeThreshold = TSFileConfig.pageSizeInByte;
-        ChunkWriterImpl seriesWriterImpl = new ChunkWriterImpl(measurementSchema, pageWriter,
-            pageSizeThreshold);
-        // write the series data
-        writeOneSeries(path.getDevice(), seriesWriterImpl, dataType,
-            seriesReader,
-            startTimeMap, endTimeMap, timeValuePair);
-        // flush the series data
-        seriesWriterImpl.writeToFileWriter(mergeFileWriter);
-      }
-    } finally {
-      for (OverflowInsertFile overflowInsertFile : overflowSeriesDataSource
-          .getOverflowInsertFileList()) {
-        FileReaderManager.getInstance()
-            .decreaseFileReaderReference(overflowInsertFile.getFilePath(),
-                false);
-      }
-    }
-    return numOfChunk;
-  }
-
-
-  private void writeOneSeries(String deviceId, ChunkWriterImpl seriesWriterImpl,
-      TSDataType dataType, IReader seriesReader, Map<String, Long> startTimeMap,
-      Map<String, Long> endTimeMap, TimeValuePair firstTVPair) throws IOException {
-    long startTime;
-    long endTime;
-    TimeValuePair localTV = firstTVPair;
-    writeTVPair(seriesWriterImpl, dataType, localTV);
-    startTime = endTime = localTV.getTimestamp();
-    if (!startTimeMap.containsKey(deviceId) || startTimeMap.get(deviceId) > startTime) {
-      startTimeMap.put(deviceId, startTime);
-    }
-    if (!endTimeMap.containsKey(deviceId) || endTimeMap.get(deviceId) < endTime) {
-      endTimeMap.put(deviceId, endTime);
-    }
-    while (seriesReader.hasNext()) {
-      localTV = seriesReader.next();
-      endTime = localTV.getTimestamp();
-      writeTVPair(seriesWriterImpl, dataType, localTV);
-    }
-    if (!endTimeMap.containsKey(deviceId) || endTimeMap.get(deviceId) < endTime) {
-      endTimeMap.put(deviceId, endTime);
-    }
-  }
-
-  private void writeTVPair(ChunkWriterImpl seriesWriterImpl, TSDataType dataType,
-      TimeValuePair timeValuePair) throws IOException {
-    switch (dataType) {
-      case BOOLEAN:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBoolean());
-        break;
-      case INT32:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getInt());
-        break;
-      case INT64:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getLong());
-        break;
-      case FLOAT:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getFloat());
-        break;
-      case DOUBLE:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getDouble());
-        break;
-      case TEXT:
-        seriesWriterImpl.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBinary());
-        break;
-      default:
-        LOGGER.error("Not support data type: {}", dataType);
-        break;
-    }
-  }
-
-
-  private String constructOutputFilePath(String baseDir, String processorName, String fileName) {
-
-    String localBaseDir = baseDir;
-    if (localBaseDir.charAt(localBaseDir.length() - 1) != File.separatorChar) {
-      localBaseDir = localBaseDir + File.separatorChar + processorName;
-    }
-    File dataDir = new File(localBaseDir);
-    if (!dataDir.exists()) {
-      LOGGER.warn("The bufferwrite processor data dir doesn't exists, create new directory {}",
-          localBaseDir);
-      dataDir.mkdirs();
-    }
-    File outputFile = new File(dataDir, fileName);
-    return outputFile.getPath();
-  }
-
-  private FileSchema constructFileSchema(String processorName) throws WriteProcessException {
-
-    List<MeasurementSchema> columnSchemaList;
-    columnSchemaList = mManager.getSchemaForFileName(processorName);
-
-    FileSchema schema = new FileSchema();
-    for (MeasurementSchema measurementSchema : columnSchemaList) {
-      schema.registerMeasurement(measurementSchema);
-    }
-    return schema;
-
-  }
-
-  @Override
-  public boolean canBeClosed() {
-    if (isMerging != FileNodeProcessorStatus.NONE) {
-      LOGGER.info("The filenode {} can't be closed, because the filenode status is {}",
-          getProcessorName(),
-          isMerging);
-      return false;
-    }
-    if (newMultiPassCount.get() != 0) {
-      LOGGER.warn("The filenode {} can't be closed, because newMultiPassCount is {}. The newMultiPassTokenSet is {}",
-          getProcessorName(), newMultiPassCount, newMultiPassTokenSet);
-      return false;
-    }
-
-    if (oldMultiPassCount == null) {
-      return true;
-    }
-    if (oldMultiPassCount.getCount() == 0) {
-      return true;
-    } else {
-      LOGGER.info("The filenode {} can't be closed, because oldMultiPassCount is {}",
-          getProcessorName(), oldMultiPassCount.getCount());
-      return false;
-    }
-  }
-
-  @Override
-  public FileNodeFlushFuture flush() throws IOException {
-    Future<Boolean> bufferWriteFlushFuture = null;
-    Future<Boolean> overflowFlushFuture = null;
-    if (bufferWriteProcessor != null && !bufferWriteProcessor.isClosed()) {
-      bufferWriteFlushFuture = bufferWriteProcessor.flush();
-    }
-    if (overflowProcessor != null && !overflowProcessor.isClosed()) {
-      overflowFlushFuture = overflowProcessor.flush();
-    }
-    return new FileNodeFlushFuture(bufferWriteFlushFuture, overflowFlushFuture);
-  }
-
-  /**
-   * Close the bufferwrite processor.
-   */
-  public void closeBufferWrite() throws FileNodeProcessorException {
-    if (bufferWriteProcessor == null || bufferWriteProcessor.isClosed()) {
-      return;
-    }
-    try {
-      while (!bufferWriteProcessor.canBeClosed()) {
-        waitForBufferWriteClose();
-      }
-      bufferWriteProcessor.close();
-      bufferWriteProcessor = null;
-    } catch (BufferWriteProcessorException e) {
-      throw new FileNodeProcessorException(e);
-    }
-  }
-
-  private void waitForBufferWriteClose() {
-    try {
-      LOGGER.info("The bufferwrite {} can't be closed, wait 100ms",
-          bufferWriteProcessor.getProcessorName());
-      TimeUnit.MICROSECONDS.sleep(100);
-    } catch (InterruptedException e) {
-      LOGGER.error("Unexpected interruption", e);
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  /**
-   * Close the overflow processor.
-   */
-  public void closeOverflow() throws FileNodeProcessorException {
-    if (overflowProcessor == null || overflowProcessor.isClosed()) {
-      return;
-    }
-    try {
-      while (!overflowProcessor.canBeClosed()) {
-        waitForOverflowClose();
-      }
-      overflowProcessor.close();
-    } catch (OverflowProcessorException e) {
-      throw new FileNodeProcessorException(e);
-    }
-  }
-
-  private void waitForOverflowClose() {
-    try {
-      LOGGER.info("The overflow {} can't be closed, wait 100ms",
-          overflowProcessor.getProcessorName());
-      TimeUnit.MICROSECONDS.sleep(100);
-    } catch (InterruptedException e) {
-      LOGGER.error("Unexpected interruption", e);
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  @Override
-  public void close() throws FileNodeProcessorException {
-    closeBufferWrite();
-    closeOverflow();
-    for (TsFileResource fileNode : newFileNodes) {
-      if (fileNode.getModFile() != null) {
-        try {
-          fileNode.getModFile().close();
-        } catch (IOException e) {
-          throw new FileNodeProcessorException(e);
-        }
-      }
-    }
-  }
-
-  /**
-   * deregister the filenode processor.
-   */
-  public void delete() throws ProcessorException {
-    if (TsFileDBConf.isEnableStatMonitor()) {
-      // remove the monitor
-      LOGGER.info("Deregister the filenode processor: {} from monitor.", getProcessorName());
-      StatMonitor.getInstance().deregisterStatistics(statStorageDeltaName);
-    }
-    closeBufferWrite();
-    closeOverflow();
-    for (TsFileResource fileNode : newFileNodes) {
-      if (fileNode.getModFile() != null) {
-        try {
-          fileNode.getModFile().close();
-        } catch (IOException e) {
-          throw new FileNodeProcessorException(e);
-        }
-      }
-    }
-  }
-
-  @Override
-  public long memoryUsage() {
-    long memSize = 0;
-    if (bufferWriteProcessor != null) {
-      memSize += bufferWriteProcessor.memoryUsage();
-    }
-    if (overflowProcessor != null) {
-      memSize += overflowProcessor.memoryUsage();
-    }
-    return memSize;
-  }
-
-  private void writeStoreToDisk(FileNodeProcessorStore fileNodeProcessorStore)
-      throws FileNodeProcessorException {
-
-    synchronized (fileNodeRestoreLock) {
-      try (FileOutputStream fileOutputStream = new FileOutputStream(fileNodeRestoreFilePath)) {
-        fileNodeProcessorStore.serialize(fileOutputStream);
-        LOGGER.debug("The filenode processor {} writes restore information to the restore file",
-            getProcessorName());
-      } catch (IOException e) {
-        throw new FileNodeProcessorException(e);
-      }
-    }
-  }
-
-  private FileNodeProcessorStore readStoreFromDisk() throws FileNodeProcessorException {
-
-    synchronized (fileNodeRestoreLock) {
-      File restoreFile = new File(fileNodeRestoreFilePath);
-      if (!restoreFile.exists() || restoreFile.length() == 0) {
-        try {
-          return new FileNodeProcessorStore(false, new HashMap<>(),
-              new TsFileResource(null, false),
-              new ArrayList<>(), FileNodeProcessorStatus.NONE, 0);
-        } catch (IOException e) {
-          throw new FileNodeProcessorException(e);
-        }
-      }
-      try (FileInputStream inputStream = new FileInputStream(fileNodeRestoreFilePath)) {
-        return FileNodeProcessorStore.deSerialize(inputStream);
-      } catch (IOException e) {
-        LOGGER
-            .error("Failed to deserialize the FileNodeRestoreFile {}, {}", fileNodeRestoreFilePath,
-                e);
-        throw new FileNodeProcessorException(e);
-      }
-    }
-  }
-
-  String getFileNodeRestoreFilePath() {
-    return fileNodeRestoreFilePath;
-  }
-
-  /**
-   * Delete data whose timestamp <= 'timestamp' and belong to timeseries deviceId.measurementId.
-   *
-   * @param deviceId the deviceId of the timeseries to be deleted.
-   * @param measurementId the measurementId of the timeseries to be deleted.
-   * @param timestamp the delete range is (0, timestamp].
-   */
-  public void delete(String deviceId, String measurementId, long timestamp) throws IOException {
-    // TODO: how to avoid partial deletion?
-    mergeDeleteLock.lock();
-    long version = versionController.nextVersion();
-
-    // record what files are updated so we can roll back them in case of exception
-    List<ModificationFile> updatedModFiles = new ArrayList<>();
-
-    try {
-      String fullPath = deviceId +
-          IoTDBConstant.PATH_SEPARATOR + measurementId;
-      Deletion deletion = new Deletion(fullPath, version, timestamp);
-      if (mergingModification != null) {
-        mergingModification.write(deletion);
-        updatedModFiles.add(mergingModification);
-      }
-      deleteBufferWriteFiles(deviceId, deletion, updatedModFiles);
-      // delete data in memory
-      OverflowProcessor ofProcessor = getOverflowProcessor(getProcessorName());
-      ofProcessor.delete(deviceId, measurementId, timestamp, version, updatedModFiles);
-      if (bufferWriteProcessor != null && !bufferWriteProcessor.isClosed()) {
-        bufferWriteProcessor.delete(deviceId, measurementId, timestamp);
-      }
-    } catch (Exception e) {
-      // roll back
-      for (ModificationFile modFile : updatedModFiles) {
-        modFile.abort();
-      }
-      throw new IOException(e);
-    } finally {
-      mergeDeleteLock.unlock();
-    }
-  }
-
-  private void deleteBufferWriteFiles(String deviceId, Deletion deletion,
-      List<ModificationFile> updatedModFiles) throws IOException {
-    if (currentTsFileResource != null && currentTsFileResource.containsDevice(deviceId)) {
-      currentTsFileResource.getModFile().write(deletion);
-      updatedModFiles.add(currentTsFileResource.getModFile());
-    }
-    for (TsFileResource fileNode : newFileNodes) {
-      if (fileNode != currentTsFileResource && fileNode.containsDevice(deviceId)
-          && fileNode.getStartTime(deviceId) <= deletion.getTimestamp()) {
-        fileNode.getModFile().write(deletion);
-        updatedModFiles.add(fileNode.getModFile());
-      }
-    }
-  }
-
-  /**
-   * Similar to delete(), but only deletes data in BufferWrite. Only used by WAL recovery.
-   */
-  public void deleteBufferWrite(String deviceId, String measurementId, long timestamp)
-      throws IOException, BufferWriteProcessorException {
-    String fullPath = deviceId +
-        IoTDBConstant.PATH_SEPARATOR + measurementId;
-    long version = versionController.nextVersion();
-    Deletion deletion = new Deletion(fullPath, version, timestamp);
-
-    List<ModificationFile> updatedModFiles = new ArrayList<>();
-    try {
-      deleteBufferWriteFiles(deviceId, deletion, updatedModFiles);
-    } catch (IOException e) {
-      for (ModificationFile modificationFile : updatedModFiles) {
-        modificationFile.abort();
-      }
-      throw e;
-    }
-    if (bufferWriteProcessor != null && !bufferWriteProcessor.isClosed()) {
-      try {
-        bufferWriteProcessor.delete(deviceId, measurementId, timestamp);
-      } catch (BufferWriteProcessorException e) {
-        throw new IOException(e);
-      }
-    }
-  }
-
-  /**
-   * Similar to delete(), but only deletes data in Overflow. Only used by WAL recovery.
-   */
-  public void deleteOverflow(String deviceId, String measurementId, long timestamp)
-      throws IOException {
-    long version = versionController.nextVersion();
-
-    OverflowProcessor overflowProcessor = getOverflowProcessor(getProcessorName());
-    List<ModificationFile> updatedModFiles = new ArrayList<>();
-    try {
-      overflowProcessor.delete(deviceId, measurementId, timestamp, version, updatedModFiles);
-    } catch (IOException e) {
-      for (ModificationFile modificationFile : updatedModFiles) {
-        modificationFile.abort();
-      }
-      throw e;
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    if (!super.equals(o)) {
-      return false;
-    }
-    FileNodeProcessor that = (FileNodeProcessor) o;
-    return isOverflowed == that.isOverflowed &&
-        numOfMergeFile == that.numOfMergeFile &&
-        lastMergeTime == that.lastMergeTime &&
-        shouldRecovery == that.shouldRecovery &&
-        multiPassLockToken == that.multiPassLockToken &&
-        Objects.equals(statStorageDeltaName, that.statStorageDeltaName) &&
-        Objects.equals(statParamsHashMap, that.statParamsHashMap) &&
-        Objects.equals(lastUpdateTimeMap, that.lastUpdateTimeMap) &&
-        Objects.equals(flushLastUpdateTimeMap, that.flushLastUpdateTimeMap) &&
-        Objects.equals(invertedIndexOfFiles, that.invertedIndexOfFiles) &&
-        Objects.equals(emptyTsFileResource, that.emptyTsFileResource) &&
-        Objects.equals(currentTsFileResource, that.currentTsFileResource) &&
-        Objects.equals(newFileNodes, that.newFileNodes) &&
-        isMerging == that.isMerging &&
-        Objects.equals(fileNodeProcessorStore, that.fileNodeProcessorStore) &&
-        Objects.equals(fileNodeRestoreFilePath, that.fileNodeRestoreFilePath) &&
-        Objects.equals(bufferWriteProcessor, that.bufferWriteProcessor) &&
-        Objects.equals(overflowProcessor, that.overflowProcessor) &&
-        Objects.equals(oldMultiPassTokenSet, that.oldMultiPassTokenSet) &&
-        Objects.equals(newMultiPassTokenSet, that.newMultiPassTokenSet) &&
-        Objects.equals(oldMultiPassCount, that.oldMultiPassCount) &&
-        Objects.equals(newMultiPassCount, that.newMultiPassCount) &&
-        Objects.equals(parameters, that.parameters) &&
-        Objects.equals(fileSchema, that.fileSchema) &&
-        Objects.equals(flushFileNodeProcessorAction, that.flushFileNodeProcessorAction) &&
-        Objects.equals(bufferwriteFlushAction, that.bufferwriteFlushAction) &&
-        Objects.equals(bufferwriteCloseAction, that.bufferwriteCloseAction) &&
-        Objects.equals(overflowFlushAction, that.overflowFlushAction);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(super.hashCode(), statStorageDeltaName, statParamsHashMap, isOverflowed,
-        lastUpdateTimeMap, flushLastUpdateTimeMap, invertedIndexOfFiles,
-        emptyTsFileResource, currentTsFileResource, newFileNodes, isMerging,
-        numOfMergeFile, fileNodeProcessorStore, fileNodeRestoreFilePath,
-        lastMergeTime, bufferWriteProcessor, overflowProcessor, oldMultiPassTokenSet,
-        newMultiPassTokenSet, oldMultiPassCount, newMultiPassCount, shouldRecovery, parameters,
-        fileSchema, flushFileNodeProcessorAction, bufferwriteFlushAction,
-        bufferwriteCloseAction, overflowFlushAction, multiPassLockToken);
-  }
-
-  public class MergeRunnale implements Runnable {
-
-    @Override
-    public void run() {
-      try {
-        ZoneId zoneId = IoTDBDescriptor.getInstance().getConfig().getZoneID();
-        long mergeStartTime = System.currentTimeMillis();
-        writeLock();
-        merge();
-        long mergeEndTime = System.currentTimeMillis();
-        long intervalTime = mergeEndTime - mergeStartTime;
-        LOGGER.info(
-            "The filenode processor {} merge start time is {}, "
-                + "merge end time is {}, merge consumes {}ms.",
-            getProcessorName(), ofInstant(Instant.ofEpochMilli(mergeStartTime),
-                zoneId), ofInstant(Instant.ofEpochMilli(mergeEndTime),
-                zoneId), intervalTime);
-      } catch (FileNodeProcessorException | DiskSpaceInsufficientException e) {
-        LOGGER.error("The filenode processor {} encountered an error when merging.",
-            getProcessorName(), e);
-        throw new ErrorDebugException(e);
-      }
-    }
-  }
-}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStatus.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStatus.java
deleted file mode 100644
index b3ce1e5..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStatus.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-public enum FileNodeProcessorStatus {
-  NONE, MERGING_WRITE, WAITING;
-
-  public static FileNodeProcessorStatus deserialize(short i) {
-    switch (i) {
-      case 0:
-        return NONE;
-      case 1:
-        return MERGING_WRITE;
-      case 2:
-        return WAITING;
-      default:
-        throw new IllegalArgumentException(
-            String.format("Invalid input %d for FileNodeProcessorStatus", i));
-    }
-  }
-
-  public short serialize() {
-    switch (this) {
-      case NONE:
-        return 0;
-      case MERGING_WRITE:
-        return 1;
-      case WAITING:
-        return 2;
-      default:
-        throw new IllegalStateException("Unsupported type");
-    }
-
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStore.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStore.java
deleted file mode 100644
index 7c1eca4..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessorStore.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * FileNodeProcessorStore is used to store information about FileNodeProcessor's status.
- * lastUpdateTime is changed and stored by BufferWrite flush or BufferWrite close.
- * emptyTsFileResource and newFileNodes are changed and stored by Overflow flush and
- * Overflow close. fileNodeProcessorState is changed and stored by the change of FileNodeProcessor's
- * status such as "work->merge merge->wait wait->work". numOfMergeFile is changed
- * and stored when FileNodeProcessor's status changes from work to merge.
- */
-public class FileNodeProcessorStore implements Serializable {
-
-  private static final long serialVersionUID = -54525372941897565L;
-
-  private boolean isOverflowed;
-  private Map<String, Long> lastUpdateTimeMap;
-  private TsFileResource emptyTsFileResource;
-  private List<TsFileResource> newFileNodes;
-  private int numOfMergeFile;
-  private FileNodeProcessorStatus fileNodeProcessorStatus;
-
-  /**
-   * Constructor of FileNodeProcessorStore.
-   *
-   * @param isOverflowed whether this FileNode contains unmerged Overflow operations.
-   * @param lastUpdateTimeMap the timestamp of last data point of each device in this FileNode.
-   * @param emptyTsFileResource a place holder when the FileNode contains no TsFile.
-   * @param newFileNodes TsFiles in the FileNode.
-   * @param fileNodeProcessorStatus the status of the FileNode.
-   * @param numOfMergeFile the number of files already merged in one merge operation.
-   */
-  public FileNodeProcessorStore(boolean isOverflowed, Map<String, Long> lastUpdateTimeMap,
-      TsFileResource emptyTsFileResource,
-      List<TsFileResource> newFileNodes,
-      FileNodeProcessorStatus fileNodeProcessorStatus,
-      int numOfMergeFile) {
-    this.isOverflowed = isOverflowed;
-    this.lastUpdateTimeMap = lastUpdateTimeMap;
-    this.emptyTsFileResource = emptyTsFileResource;
-    this.newFileNodes = newFileNodes;
-    this.fileNodeProcessorStatus = fileNodeProcessorStatus;
-    this.numOfMergeFile = numOfMergeFile;
-  }
-
-  public void serialize(OutputStream outputStream) throws IOException {
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    ReadWriteIOUtils.write(this.isOverflowed, byteArrayOutputStream);
-    // lastUpdateTimeMap
-    ReadWriteIOUtils.write(lastUpdateTimeMap.size(), byteArrayOutputStream);
-    for (Entry<String, Long> entry : lastUpdateTimeMap.entrySet()) {
-      ReadWriteIOUtils.write(entry.getKey(), byteArrayOutputStream);
-      ReadWriteIOUtils.write(entry.getValue(), byteArrayOutputStream);
-    }
-    this.emptyTsFileResource.serialize(byteArrayOutputStream);
-    ReadWriteIOUtils.write(this.newFileNodes.size(), byteArrayOutputStream);
-    for (TsFileResource tsFileResource : this.newFileNodes) {
-      tsFileResource.serialize(byteArrayOutputStream);
-    }
-    ReadWriteIOUtils.write(this.numOfMergeFile, byteArrayOutputStream);
-    ReadWriteIOUtils.write(this.fileNodeProcessorStatus.serialize(), byteArrayOutputStream);
-    // buffer array to outputstream
-    byteArrayOutputStream.writeTo(outputStream);
-  }
-
-  public static FileNodeProcessorStore deSerialize(InputStream inputStream) throws IOException {
-    boolean isOverflowed = ReadWriteIOUtils.readBool(inputStream);
-    Map<String, Long> lastUpdateTimeMap = new HashMap<>();
-    int size = ReadWriteIOUtils.readInt(inputStream);
-    for (int i = 0; i < size; i++) {
-      String path = ReadWriteIOUtils.readString(inputStream);
-      long time = ReadWriteIOUtils.readLong(inputStream);
-      lastUpdateTimeMap.put(path, time);
-    }
-    TsFileResource emptyTsFileResource = TsFileResource.deSerialize(inputStream);
-    size = ReadWriteIOUtils.readInt(inputStream);
-    List<TsFileResource> newFileNodes = new ArrayList<>();
-    for (int i = 0; i < size; i++) {
-      newFileNodes.add(TsFileResource.deSerialize(inputStream));
-    }
-    int numOfMergeFile = ReadWriteIOUtils.readInt(inputStream);
-    FileNodeProcessorStatus fileNodeProcessorStatus = FileNodeProcessorStatus
-        .deserialize(ReadWriteIOUtils.readShort(inputStream));
-
-    return new FileNodeProcessorStore(isOverflowed, lastUpdateTimeMap, emptyTsFileResource,
-        newFileNodes, fileNodeProcessorStatus, numOfMergeFile);
-  }
-
-  public boolean isOverflowed() {
-    return isOverflowed;
-  }
-
-  public void setOverflowed(boolean isOverflowed) {
-    this.isOverflowed = isOverflowed;
-  }
-
-  public FileNodeProcessorStatus getFileNodeProcessorStatus() {
-    return fileNodeProcessorStatus;
-  }
-
-  public void setFileNodeProcessorStatus(FileNodeProcessorStatus fileNodeProcessorStatus) {
-    this.fileNodeProcessorStatus = fileNodeProcessorStatus;
-  }
-
-  public Map<String, Long> getLastUpdateTimeMap() {
-    return new HashMap<>(lastUpdateTimeMap);
-  }
-
-  public void setLastUpdateTimeMap(Map<String, Long> lastUpdateTimeMap) {
-    this.lastUpdateTimeMap = lastUpdateTimeMap;
-  }
-
-  public TsFileResource getEmptyTsFileResource() {
-    return emptyTsFileResource;
-  }
-
-  public void setEmptyTsFileResource(TsFileResource emptyTsFileResource) {
-    this.emptyTsFileResource = emptyTsFileResource;
-  }
-
-  public List<TsFileResource> getNewFileNodes() {
-    return newFileNodes;
-  }
-
-  public void setNewFileNodes(List<TsFileResource> newFileNodes) {
-    this.newFileNodes = newFileNodes;
-  }
-
-  public int getNumOfMergeFile() {
-    return numOfMergeFile;
-  }
-
-  public void setNumOfMergeFile(int numOfMergeFile) {
-    this.numOfMergeFile = numOfMergeFile;
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/OverflowChangeType.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/OverflowChangeType.java
deleted file mode 100644
index 48c8eee..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/OverflowChangeType.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-/**
- * if a file is not changed by overflow, it's in NO_CHANGE;<br>
- * if it's changed and in NO_CHANGE previously, NO_CHANGE-->CHANGED, update file<br>
- * If it's changed and in CHANGED previously, and in merging, CHANGED-->MERGING_CHANGE, update file<br>
- * If it's changed and in CHANGED previously, and not in merging, do nothing<br>
- * After merging, if it's MERGING_CHANGE, MERGING_CHANGE-->CHANGED, otherwise in NO_CHANGE, MERGING_CHANGE-->NO_CHANGE
- */
-public enum OverflowChangeType {
-  NO_CHANGE, CHANGED, MERGING_CHANGE;
-
-  public short serialize() {
-    switch (this) {
-      case NO_CHANGE:
-        return 0;
-      case CHANGED:
-        return 1;
-      case MERGING_CHANGE:
-        return 2;
-      default:
-        throw new IllegalStateException("Unsupported type");
-    }
-  }
-
-  public static OverflowChangeType deserialize(short i) {
-    switch (i) {
-      case 0:
-        return NO_CHANGE;
-      case 1:
-        return CHANGED;
-      case 2:
-        return MERGING_CHANGE;
-      default:
-        throw new IllegalArgumentException(
-            String.format("Invalid input %d for OverflowChangeType", i));
-    }
-  }
-}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TsFileResource.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TsFileResource.java
deleted file mode 100644
index 3d6d74a..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TsFileResource.java
+++ /dev/null
@@ -1,390 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.filenode;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.locks.ReadWriteLock;
-import org.apache.iotdb.db.conf.directories.Directories;
-import org.apache.iotdb.db.engine.bufferwrite.RestorableTsFileIOWriter;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadataIndex;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * This class is used to store one bufferwrite file status.<br>
- */
-public class TsFileResource {
-
-  private OverflowChangeType overflowChangeType;
-
-  //the file index of `settled` folder in the Directories.
-  private int baseDirIndex;
-  private File file;
-  private Map<String, Long> startTimeMap;
-  private Map<String, Long> endTimeMap;
-  private Set<String> mergeChanged = new HashSet<>();
-
-  private transient ModificationFile modFile;
-
-  /**
-   * @param autoRead whether read the file to initialize startTimeMap and endTimeMap
-   */
-  public TsFileResource(File file, boolean autoRead) throws IOException {
-    this(new HashMap<>(), new HashMap<>(), OverflowChangeType.NO_CHANGE, file);
-    if (autoRead) {
-      //init startTime and endTime
-      try (TsFileSequenceReader reader = new TsFileSequenceReader(file.getAbsolutePath())) {
-        if (reader.readTailMagic().equals(TSFileConfig.MAGIC_STRING)) {
-          //this is a complete tsfile, and we can read the metadata directly.
-          for (Map.Entry<String, TsDeviceMetadataIndex> deviceEntry : reader.readFileMetadata()
-              .getDeviceMap().entrySet()) {
-            startTimeMap.put(deviceEntry.getKey(), deviceEntry.getValue().getStartTime());
-            endTimeMap.put(deviceEntry.getKey(), deviceEntry.getValue().getEndTime());
-          }
-        } else {
-          //sadly, this is not a complete tsfile. we have to repair it bytes by bytes
-          //TODO will implement it
-          List<ChunkGroupMetaData> metaDataList = new ArrayList<>();
-          reader.selfCheck(null, metaDataList, false);
-          initTimeMapFromChunGroupMetaDatas(metaDataList);
-        }
-      }
-    }
-  }
-
-  /**
-   * @param writer an unclosed TsFile Writer
-   */
-  public TsFileResource(File file, RestorableTsFileIOWriter writer) {
-    this(new HashMap<>(), new HashMap<>(), OverflowChangeType.NO_CHANGE, file);
-    initTimeMapFromChunGroupMetaDatas(writer.getChunkGroupMetaDatas());
-  }
-
-  private void initTimeMapFromChunGroupMetaDatas(List<ChunkGroupMetaData> metaDataList) {
-    for (ChunkGroupMetaData metaData : metaDataList) {
-      long startTime = startTimeMap.getOrDefault(metaData.getDeviceID(), Long.MAX_VALUE);
-      long endTime = endTimeMap.getOrDefault(metaData.getDeviceID(), Long.MIN_VALUE);
-      for (ChunkMetaData chunk : metaData.getChunkMetaDataList()) {
-        if (chunk.getStartTime() < startTime) {
-          startTime = chunk.getStartTime();
-        }
-        if (chunk.getEndTime() > endTime) {
-          endTime = chunk.getEndTime();
-        }
-      }
-      startTimeMap.put(metaData.getDeviceID(), startTime);
-      endTimeMap.put(metaData.getDeviceID(), endTime);
-    }
-  }
-
-
-  public TsFileResource(Map<String, Long> startTimeMap, Map<String, Long> endTimeMap,
-      OverflowChangeType type, File file) {
-
-    this.overflowChangeType = type;
-    if (file != null) {
-      this.baseDirIndex = Directories.getInstance()
-          .getTsFileFolderIndex(file.getParentFile().getParent());
-      this.modFile = new ModificationFile(file.getAbsolutePath() + ModificationFile.FILE_SUFFIX);
-    }
-    this.file = file;
-
-    this.startTimeMap = startTimeMap;
-    this.endTimeMap = endTimeMap;
-
-  }
-
-  public void serialize(OutputStream outputStream) throws IOException {
-    ReadWriteIOUtils.write(this.overflowChangeType.serialize(), outputStream);
-    ReadWriteIOUtils.write(this.baseDirIndex, outputStream);
-    ReadWriteIOUtils.writeIsNull(this.file, outputStream);
-    if (this.file != null) {
-      ReadWriteIOUtils.write(getRelativePath(), outputStream);
-    }
-    ReadWriteIOUtils.write(this.startTimeMap.size(), outputStream);
-    for (Entry<String, Long> entry : this.startTimeMap.entrySet()) {
-      ReadWriteIOUtils.write(entry.getKey(), outputStream);
-      ReadWriteIOUtils.write(entry.getValue(), outputStream);
-    }
-    ReadWriteIOUtils.write(this.endTimeMap.size(), outputStream);
-    for (Entry<String, Long> entry : this.endTimeMap.entrySet()) {
-      ReadWriteIOUtils.write(entry.getKey(), outputStream);
-      ReadWriteIOUtils.write(entry.getValue(), outputStream);
-    }
-    ReadWriteIOUtils.write(mergeChanged.size(), outputStream);
-    for (String mergeChangedElement : this.mergeChanged) {
-      ReadWriteIOUtils.write(mergeChangedElement, outputStream);
-    }
-  }
-
-  public static TsFileResource deSerialize(InputStream inputStream) throws IOException {
-    OverflowChangeType overflowChangeType = OverflowChangeType
-        .deserialize(ReadWriteIOUtils.readShort(inputStream));
-    int baseDirIndex = ReadWriteIOUtils.readInt(inputStream);
-    boolean hasRelativePath = ReadWriteIOUtils.readIsNull(inputStream);
-
-    File file = null;
-    if (hasRelativePath) {
-      String relativePath = ReadWriteIOUtils.readString(inputStream);
-      file = new File(Directories.getInstance().getTsFileFolder(baseDirIndex), relativePath);
-    }
-    int size = ReadWriteIOUtils.readInt(inputStream);
-    Map<String, Long> startTimes = new HashMap<>();
-    for (int i = 0; i < size; i++) {
-      String path = ReadWriteIOUtils.readString(inputStream);
-      long time = ReadWriteIOUtils.readLong(inputStream);
-      startTimes.put(path, time);
-    }
-    size = ReadWriteIOUtils.readInt(inputStream);
-    Map<String, Long> endTimes = new HashMap<>();
-    for (int i = 0; i < size; i++) {
-      String path = ReadWriteIOUtils.readString(inputStream);
-      long time = ReadWriteIOUtils.readLong(inputStream);
-      endTimes.put(path, time);
-    }
-    size = ReadWriteIOUtils.readInt(inputStream);
-    Set<String> mergeChanaged = new HashSet<>();
-    for (int i = 0; i < size; i++) {
-      String path = ReadWriteIOUtils.readString(inputStream);
-      mergeChanaged.add(path);
-    }
-    TsFileResource tsFileResource = new TsFileResource(startTimes, endTimes, overflowChangeType, file);
-    tsFileResource.mergeChanged = mergeChanaged;
-    return tsFileResource;
-  }
-
-
-  public void setStartTime(String deviceId, long startTime) {
-
-    startTimeMap.put(deviceId, startTime);
-  }
-
-  public long getStartTime(String deviceId) {
-
-    if (startTimeMap.containsKey(deviceId)) {
-      return startTimeMap.get(deviceId);
-    } else {
-      return -1;
-    }
-  }
-
-  public void removeStartTime(String deviceId) {
-    startTimeMap.remove(deviceId);
-  }
-
-  public Map<String, Long> getStartTimeMap() {
-
-    return startTimeMap;
-  }
-
-  public void setStartTimeMap(Map<String, Long> startTimeMap) {
-
-    this.startTimeMap = startTimeMap;
-  }
-
-  public void setEndTime(String deviceId, long timestamp) {
-
-    this.endTimeMap.put(deviceId, timestamp);
-  }
-
-  public long getEndTime(String deviceId) {
-
-    if (endTimeMap.get(deviceId) == null) {
-      return -1;
-    }
-    return endTimeMap.get(deviceId);
-  }
-
-  public Map<String, Long> getEndTimeMap() {
-
-    return endTimeMap;
-  }
-
-  public void setEndTimeMap(Map<String, Long> endTimeMap) {
-
-    this.endTimeMap = endTimeMap;
-  }
-
-  public void removeTime(String deviceId) {
-
-    startTimeMap.remove(deviceId);
-    endTimeMap.remove(deviceId);
-  }
-
-
-  public File getFile() {
-    return file;
-  }
-
-
-  public int getBaseDirIndex() {
-    return baseDirIndex;
-  }
-
-  public boolean checkEmpty() {
-
-    return startTimeMap.isEmpty() && endTimeMap.isEmpty();
-  }
-
-  public void clear() {
-    startTimeMap.clear();
-    endTimeMap.clear();
-    mergeChanged.clear();
-    overflowChangeType = OverflowChangeType.NO_CHANGE;
-  }
-
-  public void changeTypeToChanged(FileNodeProcessorStatus fileNodeProcessorState) {
-
-    if (fileNodeProcessorState == FileNodeProcessorStatus.MERGING_WRITE) {
-      overflowChangeType = OverflowChangeType.MERGING_CHANGE;
-    } else {
-      overflowChangeType = OverflowChangeType.CHANGED;
-    }
-  }
-
-  public void addMergeChanged(String deviceId) {
-
-    mergeChanged.add(deviceId);
-  }
-
-  public Set<String> getMergeChanged() {
-
-    return mergeChanged;
-  }
-
-  public void clearMergeChanged() {
-
-    mergeChanged.clear();
-  }
-
-  public boolean isClosed() {
-
-    return !endTimeMap.isEmpty();
-
-  }
-
-  public TsFileResource backUp() {
-
-    Map<String, Long> startTimeMapCopy = new HashMap<>(this.startTimeMap);
-    Map<String, Long> endTimeMapCopy = new HashMap<>(this.endTimeMap);
-    return new TsFileResource(startTimeMapCopy,
-        endTimeMapCopy, overflowChangeType, file);
-  }
-
-  public Set<String> getDevices() {
-    return this.startTimeMap.keySet();
-  }
-
-  @Override
-  public int hashCode() {
-
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((endTimeMap == null) ? 0 : endTimeMap.hashCode());
-    result = prime * result + ((file == null) ? 0 : file.hashCode());
-    result = prime * result + ((overflowChangeType == null) ? 0 : overflowChangeType.hashCode());
-    result = prime * result + ((startTimeMap == null) ? 0 : startTimeMap.hashCode());
-    return result;
-  }
-
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (!(o instanceof TsFileResource)) {
-      return false;
-    }
-    TsFileResource that = (TsFileResource) o;
-    return baseDirIndex == that.baseDirIndex &&
-        overflowChangeType == that.overflowChangeType &&
-        Objects.equals(file, that.file) &&
-        Objects.equals(startTimeMap, that.startTimeMap) &&
-        Objects.equals(endTimeMap, that.endTimeMap) &&
-        Objects.equals(mergeChanged, that.mergeChanged) &&
-        Objects.equals(modFile, that.modFile);
-  }
-
-
-  public OverflowChangeType getOverflowChangeType() {
-    return overflowChangeType;
-  }
-
-  public void setOverflowChangeType(OverflowChangeType overflowChangeType) {
-    this.overflowChangeType = overflowChangeType;
-  }
-
-  public synchronized ModificationFile getModFile() {
-    if (modFile == null) {
-      modFile = new ModificationFile(file.getAbsolutePath() + ModificationFile.FILE_SUFFIX);
-    }
-    return modFile;
-  }
-
-  public boolean containsDevice(String deviceId) {
-    return startTimeMap.containsKey(deviceId);
-  }
-
-  public void setModFile(ModificationFile modFile) {
-    this.modFile = modFile;
-  }
-
-  public void close() throws IOException {
-    modFile.close();
-  }
-
-  public String getRelativePath() {
-    if (file == null) {
-      return null;
-    }
-    return this.getFile().getParentFile().getName() + File.separator + this.getFile().getName();
-  }
-
-  public void setFile(File file) throws IOException {
-    this.file = file;
-    if (file != null) {
-      this.baseDirIndex = Directories.getInstance()
-          .getTsFileFolderIndex(file.getParentFile().getParent());
-      if (this.modFile != null) {
-        this.modFile.close();
-      }
-      this.modFile = new ModificationFile(file.getAbsolutePath() + ModificationFile.FILE_SUFFIX);
-    }
-  }
-
-  public String getFilePath() {
-    return this.getFile().getAbsolutePath();
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/BasicMemController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/BasicMemController.java
deleted file mode 100644
index e490044..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/BasicMemController.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.memcontrol;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.service.IService;
-import org.apache.iotdb.db.service.ServiceType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public abstract class BasicMemController implements IService {
-
-  private static final Logger logger = LoggerFactory.getLogger(BasicMemController.class);
-  protected long warningThreshold;
-  protected long dangerouseThreshold;
-  protected MemMonitorThread monitorThread;
-  protected MemStatisticThread memStatisticThread;
-  private IoTDBConfig config;
-
-  BasicMemController(IoTDBConfig config) {
-    this.config = config;
-    warningThreshold = config.getMemThresholdWarning();
-    dangerouseThreshold = config.getMemThresholdDangerous();
-  }
-
-  /**
-   * change instance here.
-   *
-   * @return BasicMemController
-   */
-  public static BasicMemController getInstance() {
-    switch (ControllerType.values()[IoTDBDescriptor.getInstance().getConfig()
-        .getMemControllerType()]) {
-      case JVM:
-        return JVMMemController.getInstance();
-      case RECORD:
-        return RecordMemController.getInstance();
-      case DISABLED:
-      default:
-        return DisabledMemController.getInstance();
-    }
-  }
-
-  @Override
-  public void start() throws StartupException {
-    try {
-      if (config.isEnableMemMonitor()) {
-        if (monitorThread == null) {
-          monitorThread = new MemMonitorThread(config);
-          monitorThread.start();
-        } else {
-          logger.error("Attempt to start MemController but it has already started");
-        }
-        if (memStatisticThread == null) {
-          memStatisticThread = new MemStatisticThread();
-          memStatisticThread.start();
-        } else {
-          logger.warn("Attempt to start MemController but it has already started");
-        }
-      }
-      logger.info("MemController starts");
-    } catch (Exception e) {
-      throw new StartupException(e);
-    }
-
-  }
-
-  @Override
-  public void stop() {
-    clear();
-    close();
-  }
-
-  @Override
-  public ServiceType getID() {
-    return ServiceType.JVM_MEM_CONTROL_SERVICE;
-  }
-
-  /**
-   * set dangerous threshold.
-   *
-   * @param dangerouseThreshold dangerous threshold
-   */
-  public void setDangerousThreshold(long dangerouseThreshold) {
-    this.dangerouseThreshold = dangerouseThreshold;
-  }
-
-  /**
-   * set warning threshold.
-   *
-   * @param warningThreshold warning threshold
-   */
-  public void setWarningThreshold(long warningThreshold) {
-    this.warningThreshold = warningThreshold;
-  }
-
-  /**
-   * set check interval.
-   *
-   * @param checkInterval check interval
-   */
-  public void setCheckInterval(long checkInterval) {
-    if (this.monitorThread != null) {
-      this.monitorThread.setCheckInterval(checkInterval);
-    }
-  }
-
-  public abstract long getTotalUsage();
-
-  public abstract UsageLevel getCurrLevel();
-
-  public abstract void clear();
-
-  /**
-   * close MemController.
-   */
-  public void close() {
-    logger.info("MemController exiting");
-    if (monitorThread != null) {
-      monitorThread.interrupt();
-      while (monitorThread.isAlive()) {
-        monitorThread.interrupt();
-      }
-      monitorThread = null;
-    }
-
-    if (memStatisticThread != null) {
-      memStatisticThread.interrupt();
-      while (memStatisticThread.isAlive()) {
-        memStatisticThread.interrupt();
-      }
-      memStatisticThread = null;
-    }
-    logger.info("MemController exited");
-  }
-
-  /**
-   * Any object (like OverflowProcessor or BufferWriteProcessor) that wants to hold some fixed size
-   * of memory should call this method to check the returned memory usage level to decide any
-   * further actions.
-   * @param user an object that wants some memory as a buffer or anything.
-   * @param usage how many bytes does the object want.
-   * @return one of the three UsageLevels:
-   *          safe - there are still sufficient memories left, the user may go on freely and this
-   *                 usage is recorded.
-   *          warning - there is only a small amount of memories available, the user would better
-   *                    try to reduce memory usage but can still proceed and this usage is recorded.
-   *          dangerous - there is almost no memories unused, the user cannot proceed before enough
-   *                    memory usages are released and this usage is NOT recorded.
-   */
-  public abstract UsageLevel acquireUsage(Object user, long usage);
-
-  /**
-   * When the memories held by one object (like OverflowProcessor or BufferWriteProcessor) is no
-   * more useful, this object should call this method to release the memories.
-   * @param user an object that holds some memory as a buffer or anything.
-   * @param freeSize how many bytes does the object want to release.
-   */
-  public abstract void releaseUsage(Object user, long freeSize);
-
-  public enum ControllerType {
-    RECORD, JVM, DISABLED
-  }
-
-  public enum UsageLevel {
-    SAFE, WARNING, DANGEROUS
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/DisabledMemController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/DisabledMemController.java
deleted file mode 100644
index 225005c..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/DisabledMemController.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.memcontrol;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-
-/**
- * DisabledMemController is used when the overhead of memory control is too high.
- */
-public class DisabledMemController extends BasicMemController {
-
-  DisabledMemController(IoTDBConfig config) {
-    super(config);
-  }
-
-  @Override
-  public long getTotalUsage() {
-    return 0;
-  }
-
-  @Override
-  public UsageLevel getCurrLevel() {
-    return UsageLevel.SAFE;
-  }
-
-  @Override
-  public void clear() {
-
-  }
-
-  @Override
-  public UsageLevel acquireUsage(Object user, long usage) {
-    return UsageLevel.SAFE;
-  }
-
-  @Override
-  public void releaseUsage(Object user, long freeSize) {
-
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/FlushPartialPolicy.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/FlushPartialPolicy.java
deleted file mode 100644
index 31a5860..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/FlushPartialPolicy.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.memcontrol;
-
-import org.apache.iotdb.db.concurrent.ThreadName;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.filenode.FileNodeManager;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class only gives a hint to FilenodeManager that it may flush some data to avoid rush hour.
- */
-public class FlushPartialPolicy implements Policy {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(FlushPartialPolicy.class);
-  private Thread workerThread;
-  private long sleepInterval = IoTDBDescriptor.getInstance().getConfig().getSmallFlushInterval();
-
-  @Override
-  public void execute() {
-    if (LOGGER.isDebugEnabled()) {
-      LOGGER.debug("Memory reaches {}, current memory size is {}, JVM memory is {}, flushing.",
-              BasicMemController.getInstance().getCurrLevel(),
-              MemUtils.bytesCntToStr(BasicMemController.getInstance().getTotalUsage()),
-              MemUtils.bytesCntToStr(Runtime.getRuntime().totalMemory()
-                      - Runtime.getRuntime().freeMemory()));
-    }
-
-    // use a thread to avoid blocking
-    if (workerThread == null) {
-      workerThread = createWorkerThread();
-      workerThread.start();
-    } else {
-      if (workerThread.isAlive()) {
-        LOGGER.debug("Last flush is ongoing...");
-      } else {
-        workerThread = createWorkerThread();
-        workerThread.start();
-      }
-    }
-  }
-
-  private Thread createWorkerThread() {
-    return new Thread(() -> {
-      FileNodeManager.getInstance().forceFlush(BasicMemController.UsageLevel.SAFE);
-      try {
-        Thread.sleep(sleepInterval);
-      } catch (InterruptedException e) {
-        LOGGER.warn("Flush worker interrupted!", e);
-        Thread.currentThread().interrupt();
-      }
-    }, ThreadName.FLUSH_PARTIAL_POLICY.getName());
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/ForceFLushAllPolicy.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/ForceFLushAllPolicy.java
deleted file mode 100644
index 53533d9..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/ForceFLushAllPolicy.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.memcontrol;
-
-import org.apache.iotdb.db.concurrent.ThreadName;
-import org.apache.iotdb.db.engine.filenode.FileNodeManager;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ForceFLushAllPolicy implements Policy {
-
-  private static final Logger logger = LoggerFactory.getLogger(ForceFLushAllPolicy.class);
-  private Thread workerThread;
-
-  @Override
-  public void execute() {
-    if (logger.isInfoEnabled()) {
-      logger.info("Memory reaches {}, current memory size is {}, JVM memory is {}, flushing.",
-              BasicMemController.getInstance().getCurrLevel(),
-              MemUtils.bytesCntToStr(BasicMemController.getInstance().getTotalUsage()),
-              MemUtils.bytesCntToStr(Runtime.getRuntime().totalMemory()
-                      - Runtime.getRuntime().freeMemory()));
-    }
-
-    // use a thread to avoid blocking
-    if (workerThread == null) {
-      workerThread = createWorkerThread();
-      workerThread.start();
-    } else {
-      if (workerThread.isAlive()) {
-        logger.info("Last flush is ongoing...");
-      } else {
-        workerThread = createWorkerThread();
-        workerThread.start();
-      }
-    }
-  }
-
-  private Thread createWorkerThread() {
-    return new Thread(() ->
-            FileNodeManager.getInstance().forceFlush(BasicMemController.UsageLevel.DANGEROUS),
-            ThreadName.FORCE_FLUSH_ALL_POLICY.getName());
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/JVMMemController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/JVMMemController.java
deleted file mode 100644
index 31a3ef8..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/JVMMemController.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * 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 org.apache.iotdb.db.engine.memcontrol;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.utils.MemUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class JVMMemController extends BasicMemController {
-
-  private static Logger LOGGER = LoggerFactory.getLogger(JVMMemController.class);
-
-  private JVMMemController(IoTDBConfig config) {
-    super(config);
-  }
-
-  public static JVMMemController getInstance() {
-    return InstanceHolder.INSTANCE;
-  }
-
-  @Override
-  public long getTotalUsage() {
-    // memory used by non-data objects, this is used to estimate the memory used by data
-    long nonDataUsage = 0;
-    return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory() - nonDataUsage;
-  }
-
-  @Override
-  public UsageLevel getCurrLevel() {
-    long memUsage = getTotalUsage();
-    if (memUsage < warningThreshold) {
-      return UsageLevel.SAFE;
-    } else if (memUsage < dangerouseThreshold) {
-      return UsageLevel.WARNING;
-    } else {
-      return UsageLevel.DANGEROUS;
-    }
-  }
-
-  @Override
-  public void clear() {
-    // JVMMemController does not need cleaning
-  }
-
-  @Override
-  public UsageLevel acquireUsage(Object user, long usage) {
-    long memUsage = getTotalUsage() + usage;
-    if (memUsage < warningThreshold) {
-      return UsageLevel.SAFE;
-    } else if (memUsage < dangerouseThreshold) {
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("Warning Threshold : {} allocated to {}, total usage {}",
-            MemUtils.bytesCntToStr(usage),
-            user.getClass(), MemUtils.bytesCntToStr(memUsage));
-      }
-      return UsageLevel.WARNING;
-    } else {
-      if (LOGGER.isWarnEnabled()) {
-        LOGGER.warn("Memory request from {} is denied, memory usage : {}", user.getClass(),
-            MemUtils.bytesCntToStr(memUsage));
-      }
-      return UsageLevel.DANGEROUS;
-    }
-  }
-
-  @Override
-  public void releaseUsage(Object user, long freeSize) {
-    if (LOGGER.isInfoEnabled()) {
-      LOGGER.info("{} freed from {}, total usage {}", MemUtils.bytesCntToStr(freeSize),
-          user.getClass(), MemUtils.bytesCntToStr(getTotalUsage()));
-    }
-  }
-
-  private static class InstanceHolder {
-
-    private InstanceHolder() {
-    }
-
-    private static final JVMMemController INSTANCE = new JVMMemController(
-        IoTDBDescriptor.getInstance().getConfig());
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/MemMonitorThread.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/MemMonitorThread.java
deleted file mode 100644
index 05226aa..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memcontrol/MemMonitorThread.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * 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
... 38938 lines suppressed ...