Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617075649


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##########
@@ -0,0 +1,1251 @@
+/*
+ * 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.pipe.receiver.protocol.pipeconsensus;
+
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.commons.consensus.DataRegionId;
+import org.apache.iotdb.commons.consensus.index.ProgressIndex;
+import org.apache.iotdb.commons.consensus.index.ProgressIndexType;
+import 
org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType;
+import 
org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion;
+import 
org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFilePieceReq;
+import 
org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.response.PipeConsensusTransferFilePieceResp;
+import 
org.apache.iotdb.commons.pipe.connector.payload.thrift.response.PipeTransferFilePieceResp;
+import org.apache.iotdb.commons.pipe.receiver.IoTDBReceiverAgent;
+import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException;
+import org.apache.iotdb.consensus.pipe.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.PipeConsensusServerImpl;
+import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.LoadFileException;
+import 
org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq;
+import 
org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq;
+import 
org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq;
+import 
org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq;
+import 
org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq;
+import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode;
+import org.apache.iotdb.db.storageengine.StorageEngine;
+import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
+import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus;
+import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils;
+import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager;
+import 
org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.tsfile.common.constant.TsFileConstant;
+import org.apache.tsfile.read.TsFileSequenceReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+public class PipeConsensusReceiver {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusReceiver.class);
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+  private final RequestExecutor requestExecutor = new RequestExecutor();
+  private final PipeConsensus pipeConsensus;
+  private final ConsensusGroupId consensusGroupId;
+  // Used to buffer TsFile when transfer TsFile asynchronously.
+  private final List<String> receiverBaseDirsName;
+  private final TsFileDiskBufferPool tsFileDiskBufferPool = new 
TsFileDiskBufferPool();
+  private final AtomicReference<File> receiverFileDirWithIdSuffix = new 
AtomicReference<>();
+  private FolderManager folderManager;
+
+  public PipeConsensusReceiver(
+      PipeConsensus pipeConsensus,
+      ConsensusGroupId consensusGroupId,
+      ConsensusPipeName consensusPipeName) {
+    this.pipeConsensus = pipeConsensus;
+    this.consensusGroupId = consensusGroupId;
+
+    // Each pipeConsensusReceiver has its own base directories. for example, a 
default dir path is
+    // 
data/datanode/system/pipe/consensus/receiver/consensus{consensusGroupId}_{leaderDataNodeId}_{followerDataNodeId}
+    receiverBaseDirsName =
+        
Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getPipeConsensusReceiverFileDirs())
+            .map(s -> s + File.separator + consensusPipeName)
+            .collect(Collectors.toList());
+
+    try {
+      this.folderManager =
+          new FolderManager(receiverBaseDirsName, 
DirectoryStrategyType.SEQUENCE_STRATEGY);
+      initiateTsFileBufferFolder();
+    } catch (DiskSpaceInsufficientException e) {
+      LOGGER.error(
+          "Fail to create pipeConsensus receiver file folders allocation 
strategy because all disks of folders are full.",
+          e);
+    }
+  }
+
+  /**
+   * This method cannot be set to synchronize. Receive events can be 
concurrent since reqBuffer but
+   * load event must be synchronized.
+   */
+  public TPipeConsensusTransferResp receive(final TPipeConsensusTransferReq 
req) {
+    // PreCheck: if there are these cases: read-only; null impl; inactive 
impl, etc. The receiver
+    // will reject synchronization.
+    TPipeConsensusTransferResp resp = preCheckForReceiver(req);
+    if (resp != null) {
+      return resp;
+    }
+
+    final short rawRequestType = req.getType();
+    if (PipeConsensusRequestType.isValidatedRequestType(rawRequestType)) {
+      switch (PipeConsensusRequestType.valueOf(rawRequestType)) {
+        case TRANSFER_TS_FILE_PIECE:
+        case TRANSFER_TS_FILE_PIECE_WITH_MOD:
+          {
+            // Just take a place in requestExecutor's buffer, the further seal 
request will remove
+            // its place from buffer.
+            Object ignore = requestExecutor.onRequest(req, true);
+            return loadEvent(req);
+          }
+        case TRANSFER_TS_FILE_SEAL:
+        case TRANSFER_TS_FILE_SEAL_WITH_MOD:
+          // TODO: check memory when logging wal(in further version)
+        case TRANSFER_TABLET_RAW:
+        case TRANSFER_TABLET_BINARY:
+        case TRANSFER_TABLET_INSERT_NODE:
+          // TODO: support batch transfer(in further version)
+        case TRANSFER_TABLET_BATCH:
+        default:
+          return requestExecutor.onRequest(req, false);
+      }
+    }
+    // Unknown request type, which means the request can not be handled by 
this receiver,
+    // maybe the version of the receiver is not compatible with the sender
+    final TSStatus status =
+        RpcUtils.getStatus(
+            TSStatusCode.PIPE_TYPE_ERROR,
+            String.format("PipeConsensus Unknown PipeRequestType %s.", 
rawRequestType));
+    if (LOGGER.isWarnEnabled()) {
+      LOGGER.warn("PipeConsensus Unknown PipeRequestType, response status = 
{}.", status);
+    }
+    return new TPipeConsensusTransferResp(status);
+  }
+
+  private TPipeConsensusTransferResp preCheckForReceiver(final 
TPipeConsensusTransferReq req) {
+    ConsensusGroupId groupId =
+        
ConsensusGroupId.Factory.createFromTConsensusGroupId(req.getConsensusGroupId());
+    PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId);
+
+    if (impl == null) {
+      String message = String.format("PipeConsensus: unexpected 
consensusGroupId %s", groupId);
+      if (LOGGER.isErrorEnabled()) {
+        LOGGER.error(message);
+      }
+      return new TPipeConsensusTransferResp(
+          
RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode(), 
message));
+    }
+    if (impl.isReadOnly()) {
+      String message =
+          String.format(
+              "PipeConsensus-ConsensusGroupId-%s: fail to receive because 
system is read-only.",
+              groupId);
+      if (LOGGER.isErrorEnabled()) {
+        LOGGER.error(message);
+      }
+      return new TPipeConsensusTransferResp(
+          RpcUtils.getStatus(TSStatusCode.SYSTEM_READ_ONLY.getStatusCode(), 
message));
+    }
+    if (!impl.isActive()) {
+      String message =
+          String.format(
+              "PipeConsensus-ConsensusGroupId-%s: fail to receive because peer 
is inactive and not ready.",
+              groupId);
+      if (LOGGER.isWarnEnabled()) {
+        LOGGER.warn(message);
+      }
+      return new TPipeConsensusTransferResp(
+          
RpcUtils.getStatus(TSStatusCode.WRITE_PROCESS_REJECT.getStatusCode(), message));
+    }
+
+    return null;
+  }
+
+  private TPipeConsensusTransferResp loadEvent(final TPipeConsensusTransferReq 
req) {
+    // synchronized load event, ensured by upper caller's lock.
+    try {
+      final short rawRequestType = req.getType();
+      if (PipeConsensusRequestType.isValidatedRequestType(rawRequestType)) {
+        switch (PipeConsensusRequestType.valueOf(rawRequestType)) {
+          case TRANSFER_TABLET_INSERT_NODE:
+            return handleTransferTabletInsertNode(
+                
PipeConsensusTabletInsertNodeReq.fromTPipeConsensusTransferReq(req));
+          case TRANSFER_TABLET_RAW:
+            // PipeConsensus doesn't expect to handle rawTabletEvent.
+            LOGGER.error("PipeConsensus Unknown PipeRequestType: do not 
support tablet raw!");
+            return new TPipeConsensusTransferResp(
+                new 
TSStatus(TSStatusCode.PIPE_CONSENSUS_UNSUPPORTED_EVENT.getStatusCode()));
+          case TRANSFER_TABLET_BINARY:
+            return handleTransferTabletBinary(
+                
PipeConsensusTabletBinaryReq.fromTPipeConsensusTransferReq(req));
+          case TRANSFER_TS_FILE_PIECE:
+            return handleTransferFilePiece(
+                
PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), true);
+          case TRANSFER_TS_FILE_SEAL:
+            return handleTransferFileSeal(
+                PipeConsensusTsFileSealReq.fromTPipeConsensusTransferReq(req));
+          case TRANSFER_TS_FILE_PIECE_WITH_MOD:
+            return handleTransferFilePiece(
+                
PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), false);
+          case TRANSFER_TS_FILE_SEAL_WITH_MOD:
+            return handleTransferFileSealWithMods(
+                
PipeConsensusTsFileSealWithModReq.fromTPipeConsensusTransferReq(req));
+          case TRANSFER_TABLET_BATCH:
+            LOGGER.info("PipeConsensus transfer batch hasn't been implemented 
yet.");
+          default:
+            break;
+        }
+      }
+      // Unknown request type, which means the request can not be handled by 
this receiver,
+      // maybe the version of the receiver is not compatible with the sender
+      final TSStatus status =
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TYPE_ERROR,
+              String.format("Unknown PipeConsensusRequestType %s.", 
rawRequestType));
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Unknown PipeRequestType, 
response status = {}.",
+          consensusGroupId,
+          status);
+      return new TPipeConsensusTransferResp(status);
+    } catch (Exception e) {
+      final String error = String.format("Serialization error during pipe 
receiving, %s", e);
+      LOGGER.warn("PipeConsensus-ConsensusGroupId-{}: {}", consensusGroupId, 
error, e);
+      return new 
TPipeConsensusTransferResp(RpcUtils.getStatus(TSStatusCode.PIPE_ERROR, error));
+    }
+  }
+
+  private TPipeConsensusTransferResp handleTransferTabletInsertNode(
+      final PipeConsensusTabletInsertNodeReq req) throws 
ConsensusGroupNotExistException {
+    LOGGER.info(
+        "PipeConsensus-ConsensusGroupId-{}: starting to receive tablet 
insertNode",
+        consensusGroupId);
+    PipeConsensusServerImpl impl =
+        Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId))
+            .orElseThrow(() -> new 
ConsensusGroupNotExistException(consensusGroupId));
+    final InsertNode insertNode = req.getInsertNode();
+    insertNode.setProgressIndex(
+        
ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex())));
+    return new 
TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode));
+  }
+
+  private TPipeConsensusTransferResp handleTransferTabletBinary(
+      final PipeConsensusTabletBinaryReq req) throws 
ConsensusGroupNotExistException {
+    LOGGER.info(
+        "PipeConsensus-ConsensusGroupId-{}: starting to receive tablet 
binary", consensusGroupId);
+    PipeConsensusServerImpl impl =
+        Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId))
+            .orElseThrow(() -> new 
ConsensusGroupNotExistException(consensusGroupId));
+    final InsertNode insertNode = req.convertToInsertNode();
+    insertNode.setProgressIndex(
+        
ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex())));
+    return new 
TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode));
+  }
+
+  private TPipeConsensusTransferResp handleTransferFilePiece(
+      final PipeConsensusTransferFilePieceReq req, final boolean isSingleFile) 
{
+    LOGGER.info(
+        "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile 
pieces", consensusGroupId);
+    TsFileTransferDiskBuffer diskBuffer =
+        tsFileDiskBufferPool.borrowCorrespondingBuffer(req.getCommitId());
+
+    try {
+      updateWritingFileIfNeeded(diskBuffer, req.getFileName(), isSingleFile);
+      final File writingFile = diskBuffer.getWritingFile();
+      final RandomAccessFile writingFileWriter = 
diskBuffer.getWritingFileWriter();
+
+      if (isWritingFileOffsetNonCorrect(diskBuffer, 
req.getStartWritingOffset())) {
+        if (!writingFile.getName().endsWith(TsFileConstant.TSFILE_SUFFIX)) {
+          // If the file is a tsFile, then the content will not be changed for 
a specific
+          // filename. However, for other files (mod, snapshot, etc.) the 
content varies for the
+          // same name in different times, then we must rewrite the file to 
apply the newest
+          // version.
+          writingFileWriter.setLength(0);
+        }
+
+        final TSStatus status =
+            RpcUtils.getStatus(
+                TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_OFFSET_RESET,
+                String.format(
+                    "Request sender to reset file reader's offset from %s to 
%s.",
+                    req.getStartWritingOffset(), writingFileWriter.length()));
+        LOGGER.warn(
+            "PipeConsensus-ConsensusGroupId-{}: File offset reset requested by 
receiver, response status = {}.",
+            consensusGroupId,
+            status);
+        return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp(
+            status, writingFileWriter.length());
+      }
+
+      writingFileWriter.write(req.getFilePiece());
+      return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp(
+          RpcUtils.SUCCESS_STATUS, writingFileWriter.length());
+    } catch (Exception e) {
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to write file piece from 
req {}.",
+          consensusGroupId,
+          req,
+          e);
+      final TSStatus status =
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+              String.format("Failed to write file piece, because %s", 
e.getMessage()));
+      try {
+        return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp(
+            status, PipeTransferFilePieceResp.ERROR_END_OFFSET);
+      } catch (IOException ex) {
+        return 
PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp(status);
+      }
+    }
+  }
+
+  private TPipeConsensusTransferResp handleTransferFileSeal(final 
PipeConsensusTsFileSealReq req) {
+    LOGGER.info(
+        "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile seal", 
consensusGroupId);
+    TsFileTransferDiskBuffer diskBuffer =
+        tsFileDiskBufferPool.borrowCorrespondingBuffer(req.getCommitId());
+    File writingFile = diskBuffer.getWritingFile();
+    RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter();
+
+    try {
+      if (isWritingFileNonAvailable(diskBuffer)) {
+        final TSStatus status =
+            RpcUtils.getStatus(
+                TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+                String.format(
+                    "Failed to seal file, because writing file %s is not 
available.", writingFile));
+        LOGGER.warn(status.getMessage());
+        return new TPipeConsensusTransferResp(status);
+      }
+
+      final TPipeConsensusTransferResp resp =
+          checkFinalFileSeal(diskBuffer, req.getFileName(), 
req.getFileLength());
+      if (Objects.nonNull(resp)) {
+        return resp;
+      }
+
+      final String fileAbsolutePath = writingFile.getAbsolutePath();
+
+      // 1. The writing file writer must be closed, otherwise it may cause 
concurrent errors during
+      // the process of loading tsfile when parsing tsfile.
+      //
+      // 2. The writing file must be set to null, otherwise if the next passed 
tsfile has the same
+      // name as the current tsfile, it will bypass the judgment logic of
+      // updateWritingFileIfNeeded#isFileExistedAndNameCorrect, and continue 
to write to the already
+      // loaded file. Since the writing file writer has already been closed, 
it will throw a Stream
+      // Close exception.
+      writingFileWriter.close();
+      diskBuffer.setWritingFileWriter(null);
+
+      // writingFile will be deleted after load if no exception occurs
+      diskBuffer.setWritingFile(null);
+
+      final TSStatus status =
+          loadFileToDateRegion(
+              fileAbsolutePath,
+              
ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex())));
+      if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // if transfer success, disk buffer will be released.
+        diskBuffer.returnSelf();
+        LOGGER.info(
+            "PipeConsensus-ConsensusGroupId-{}: Seal file {} successfully.",
+            consensusGroupId,
+            fileAbsolutePath);
+      } else {
+        LOGGER.warn(
+            "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, 
because {}.",
+            consensusGroupId,
+            fileAbsolutePath,
+            status.getMessage());
+      }
+      return new TPipeConsensusTransferResp(status);
+    } catch (IOException e) {
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} from req 
{}.",
+          consensusGroupId,
+          writingFile,
+          req,
+          e);
+      return new TPipeConsensusTransferResp(
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+              String.format("Failed to seal file %s because %s", writingFile, 
e.getMessage())));
+    } catch (LoadFileException e) {
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to load file {} from req 
{}.",
+          consensusGroupId,
+          writingFile,
+          req,
+          e);
+      return new TPipeConsensusTransferResp(
+          RpcUtils.getStatus(
+              TSStatusCode.LOAD_FILE_ERROR,
+              String.format("Failed to seal file %s because %s", writingFile, 
e.getMessage())));
+    } finally {
+      // If the writing file is not sealed successfully, the writing file will 
be deleted.
+      // All pieces of the writing file and its mod (if exists) should be 
retransmitted by the
+      // sender.
+      closeCurrentWritingFileWriter(diskBuffer);
+      deleteCurrentWritingFile(diskBuffer);
+    }
+  }
+
+  private TPipeConsensusTransferResp handleTransferFileSealWithMods(
+      final PipeConsensusTsFileSealWithModReq req) {
+    LOGGER.info(
+        "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile seal 
with mods",
+        consensusGroupId);
+    TsFileTransferDiskBuffer diskBuffer =
+        tsFileDiskBufferPool.borrowCorrespondingBuffer(req.getCommitId());
+    File writingFile = diskBuffer.getWritingFile();
+    RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter();
+
+    final List<File> files =
+        req.getFileNames().stream()
+            .map(fileName -> new File(receiverFileDirWithIdSuffix.get(), 
fileName))
+            .collect(Collectors.toList());
+    try {
+      if (isWritingFileNonAvailable(diskBuffer)) {
+        final TSStatus status =
+            RpcUtils.getStatus(
+                TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+                String.format(
+                    "Failed to seal file %s, because writing file %s is not 
available.",
+                    req.getFileNames(), writingFile));
+        LOGGER.warn(status.getMessage());
+        return new TPipeConsensusTransferResp(status);
+      }
+
+      // Any of the transferred files cannot be empty, or else the receiver
+      // will not sense this file because no pieces are sent
+      for (int i = 0; i < req.getFileNames().size(); ++i) {
+        final TPipeConsensusTransferResp resp =
+            i == req.getFileNames().size() - 1
+                ? checkFinalFileSeal(
+                    diskBuffer, req.getFileNames().get(i), 
req.getFileLengths().get(i))
+                : checkNonFinalFileSeal(
+                    diskBuffer,
+                    files.get(i),
+                    req.getFileNames().get(i),
+                    req.getFileLengths().get(i));
+        if (Objects.nonNull(resp)) {
+          return resp;
+        }
+      }
+
+      // 1. The writing file writer must be closed, otherwise it may cause 
concurrent errors during
+      // the process of loading tsfile when parsing tsfile.
+      //
+      // 2. The writing file must be set to null, otherwise if the next passed 
tsfile has the same
+      // name as the current tsfile, it will bypass the judgment logic of
+      // updateWritingFileIfNeeded#isFileExistedAndNameCorrect, and continue 
to write to the already
+      // loaded file. Since the writing file writer has already been closed, 
it will throw a Stream
+      // Close exception.
+      writingFileWriter.close();
+      diskBuffer.setWritingFileWriter(null);
+
+      // WritingFile will be deleted after load if no exception occurs
+      diskBuffer.setWritingFile(null);
+
+      final List<String> fileAbsolutePaths =
+          
files.stream().map(File::getAbsolutePath).collect(Collectors.toList());
+
+      // only load mods
+      final TSStatus status =
+          loadFileToDateRegion(
+              fileAbsolutePaths.get(1),
+              
ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex())));
+      if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // if transfer success, disk buffer will be released.
+        diskBuffer.returnSelf();
+        LOGGER.info(
+            "PipeConsensus-ConsensusGroupId-{}: Seal file with mods {} 
successfully.",
+            consensusGroupId,
+            fileAbsolutePaths);
+      } else {
+        LOGGER.warn(
+            "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, status 
is {}.",
+            consensusGroupId,
+            fileAbsolutePaths,
+            status);
+      }
+      return new TPipeConsensusTransferResp(status);
+    } catch (Exception e) {
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} from req 
{}.",
+          consensusGroupId,
+          files,
+          req,
+          e);
+      return new TPipeConsensusTransferResp(
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+              String.format("Failed to seal file %s because %s", writingFile, 
e.getMessage())));
+    } finally {
+      // If the writing file is not sealed successfully, the writing file will 
be deleted.
+      // All pieces of the writing file and its mod(if exists) should be 
retransmitted by the
+      // sender.
+      closeCurrentWritingFileWriter(diskBuffer);
+      // Clear the directory instead of only deleting the referenced files in 
seal request
+      // to avoid previously undeleted file being redundant when transferring 
multi files
+      
IoTDBReceiverAgent.cleanPipeReceiverDir(receiverFileDirWithIdSuffix.get());
+    }
+  }
+
+  private TPipeConsensusTransferResp checkNonFinalFileSeal(
+      final TsFileTransferDiskBuffer diskBuffer,
+      final File file,
+      final String fileName,
+      final long fileLength)
+      throws IOException {
+    final RandomAccessFile writingFileWriter = 
diskBuffer.getWritingFileWriter();
+
+    if (!file.exists()) {
+      final TSStatus status =
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+              String.format("Failed to seal file %s, the file does not 
exist.", fileName));
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, because 
the file does not exist.",
+          consensusGroupId,
+          fileName);
+      return new TPipeConsensusTransferResp(status);
+    }
+
+    if (fileLength != file.length()) {
+      final TSStatus status =
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR,
+              String.format(
+                  "Failed to seal file %s, because the length of file is not 
correct. "
+                      + "The original file has length %s, but receiver file 
has length %s.",
+                  fileName, fileLength, writingFileWriter.length()));
+      LOGGER.warn(
+          "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} when 
check non final seal, because the length of file is not correct. "
+              + "The original file has length {}, but receiver file has length 
{}.",
+          consensusGroupId,
+          fileName,
+          fileLength,
+          writingFileWriter.length());
+      return new TPipeConsensusTransferResp(status);
+    }
+
+    return null;
+  }
+
+  private TSStatus loadFileToDateRegion(String filePath, ProgressIndex 
progressIndex)

Review Comment:
   fixed...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to