You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/05/14 04:11:23 UTC

[GitHub] [iotdb] fanhualta opened a new pull request #3191: New features of cluster scalability and multi-raft

fanhualta opened a new pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191


   This pr adds two new features:
   1. cluster scalability
   2. multi-raft


-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-844688923


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [127 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='48.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [48.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='2.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644975107



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -106,13 +106,13 @@ public static Long removeNode(AsyncMetaClient asyncMetaClient, Node nodeToRemove
   }
 
   public static Boolean matchTerm(
-      AsyncClient client, Node target, long prevLogIndex, long prevLogTerm, Node header)

Review comment:
       I have replaced all `Node header,int raftId` with `RaftNode raftNode`.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639873818



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -810,11 +810,11 @@ public void updatePlanIndexes(TsFileResource another) {
   }
 
   public boolean isPlanIndexOverlap(TsFileResource another) {
-    return another.maxPlanIndex >= this.minPlanIndex && another.minPlanIndex <= this.maxPlanIndex;
+    return another.maxPlanIndex > this.minPlanIndex && another.minPlanIndex < this.maxPlanIndex;
   }
 
   public boolean isPlanRangeCovers(TsFileResource another) {
-    return this.minPlanIndex <= another.minPlanIndex && another.maxPlanIndex <= this.maxPlanIndex;
+    return this.minPlanIndex < another.minPlanIndex && another.maxPlanIndex < this.maxPlanIndex;
   }

Review comment:
       I think it's necessary. When data is migrated, each file to be loaded has to reset the plan index. which will be set as max log index in log manager, so there will be multiple files with the same plan index.




-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r643918392



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)
+        throws SnapshotInstallationException {
       try {
         logger.info("Starting to install a snapshot {} into slot[{}]", snapshot, slot);
         installFileSnapshotSchema(snapshot);
         logger.info("Schemas in snapshot are registered");
-
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as the schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as the schemas are set, writes can proceed
+            slotManager.setToPullingWritable(slot);
+            logger.debug("{}: slot {} is now pulling writable", name, slot);
+          }
         }
-
-        installFileSnapshotFiles(snapshot, slot);
+        installFileSnapshotFiles(snapshot, slot, isDataMigration);
       } catch (PullFileException e) {
         throw new SnapshotInstallationException(e);
       }
     }
 
     @Override
-    public void install(Map<Integer, FileSnapshot> snapshotMap)
+    public void install(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
       logger.info("Starting to install snapshots {}", snapshotMap);
-      installSnapshot(snapshotMap);
+      installSnapshot(snapshotMap, isDataMigration);
     }
 
-    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap)
+    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
-      // ensure StorageGroups are synchronized
-      try {
-        dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(true);
-      } catch (CheckConsistencyException e) {
-        throw new SnapshotInstallationException(e);
-      }
-
-      for (FileSnapshot value : snapshotMap.values()) {
-        installFileSnapshotSchema(value);
-      }
-
+      // In data migration, meta group member other than new node does not need to synchronize the
+      // leader, because data migration must be carried out after meta group applied add/remove node
+      // log.
+      dataGroupMember
+          .getMetaGroupMember()
+          .syncLocalApply(
+              dataGroupMember.getMetaGroupMember().getPartitionTable().getLastMetaLogIndex() - 1,
+              false);
       for (Entry<Integer, FileSnapshot> integerSnapshotEntry : snapshotMap.entrySet()) {
         Integer slot = integerSnapshotEntry.getKey();
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        FileSnapshot snapshot = integerSnapshotEntry.getValue();
+        installFileSnapshotSchema(snapshot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as schemas are set, writes can proceed

Review comment:
       comment is not right




-- 
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.

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



[GitHub] [iotdb] LebronAl commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645402913



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -767,57 +781,109 @@ public void removeLocalData(List<Integer> slots) {
               SlotPartitionTable.getSlotStrategy()
                   .calculateSlotByPartitionNum(
                       storageGroupName, timePartitionId, ClusterConstant.SLOT_NUM);
+          /**
+           * If this slot is just held by different raft groups in the same node, it should keep the
+           * data of slot.
+           */
+          if (((SlotPartitionTable) metaGroupMember.getPartitionTable())
+              .judgeHoldSlot(thisNode, slot)) {
+            return false;
+          }
           return slotSet.contains(slot);
         };
     for (PartialPath sg : allStorageGroupNames) {
       StorageEngine.getInstance().removePartitions(sg, filter);
     }
     for (Integer slot : slots) {
-      slotManager.setToNull(slot);
+      slotManager.setToNull(slot, false);
     }
+    slotManager.save();
 
     if (logger.isInfoEnabled()) {
       logger.info(
           "{}: data of {} and other {} slots are removed", name, slots.get(0), slots.size() - 1);
     }
   }
 
+  public void preRemoveNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to pre remove node {}", name, removedNode);
+    }
+    synchronized (allNodes) {
+      if (allNodes.contains(removedNode) && allNodes.size() == config.getReplicationNum()) {
+        // update the group if the deleted node was in it
+        PartitionGroup newGroup = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
+        if (newGroup == null) {
+          return;
+        }
+        Node newNodeToGroup = newGroup.get(newGroup.size() - 1);
+        allNodes.add(newNodeToGroup);
+        peerMap.putIfAbsent(newNodeToGroup, new Peer(logManager.getLastLogIndex()));
+      }
+    }
+  }
+
   /**
    * When a node is removed and IT IS NOT THE HEADER of the group, the member should take over some
    * slots from the removed group, and add a new node to the group the removed node was in the
    * group.
    */
   @SuppressWarnings("java:S2445") // the reference of allNodes is unchanged
-  public void removeNode(Node removedNode, NodeRemovalResult removalResult) {
+  public void removeNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to remove node {}", name, removedNode);
+    }
+
     synchronized (allNodes) {
+      preRemoveNode(removedNode);
       if (allNodes.contains(removedNode)) {
         // update the group if the deleted node was in it
-        allNodes = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
-        initPeerMap();
+        allNodes.remove(removedNode);
+        peerMap.remove(removedNode);
         if (removedNode.equals(leader.get())) {
           // if the leader is removed, also start an election immediately
           synchronized (term) {
             setCharacter(NodeCharacter.ELECTOR);
-            setLastHeartbeatReceivedTime(Long.MIN_VALUE);
+            setLeader(null);
+          }
+          synchronized (getHeartBeatWaitObject()) {
+            getHeartBeatWaitObject().notifyAll();
           }
         }
       }
-      List<Integer> slotsToPull =
-          ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
-      if (slotsToPull != null) {
-        // pull the slots that should be taken over
-        PullSnapshotTaskDescriptor taskDescriptor =
-            new PullSnapshotTaskDescriptor(removalResult.getRemovedGroup(), slotsToPull, true);
-        pullFileSnapshot(taskDescriptor, null);
+    }
+  }
+
+  public void pullSlots(NodeRemovalResult removalResult) {
+    List<Integer> slotsToPull =
+        ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
+    if (slotsToPull != null) {
+      // pull the slots that should be taken over
+      PullSnapshotTaskDescriptor taskDescriptor =
+          new PullSnapshotTaskDescriptor(
+              removalResult.getRemovedGroup(getRaftGroupId()), new ArrayList<>(slotsToPull), true);
+      pullFileSnapshot(taskDescriptor, null);
+    }
+  }
+
+  /** For data group, it's necessary to apply remove/add log immediately after append. */
+  @Override
+  protected long appendEntry(long prevLogIndex, long prevLogTerm, long leaderCommit, Log log) {
+    long resp = super.appendEntry(prevLogIndex, prevLogTerm, leaderCommit, log);
+    if (resp == Response.RESPONSE_AGREE
+        && (log instanceof AddNodeLog || log instanceof RemoveNodeLog)) {
+      try {
+        commitLog(log);

Review comment:
       I suspect there maybe something wrong with this code, because it will commit all the logs before this addNode. Is it possible to apply only this log?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {

Review comment:
       It seems that this function will not be used?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode());

Review comment:
       What about adding a new field `public static final TSStatus NEED_REDIRECTION = getStatus(TSStatusCode. NEED_REDIRECTION); in StatusUtils`, and then try to use `StatusUtils.getStatus(StatusUtils.NEED_REDIRECTION, endPoint)`.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(

Review comment:
       It seems that this function will not be used?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {

Review comment:
       same

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {
+    TSStatus result;
+    if (entry.getValue().contains(thisNode)) {
+      // the query should be handled by a group the local node is in, handle it with in the group
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Execute {} in a local group of {}", entry.getKey(), entry.getValue().getHeader());
+      result =
+          getLocalDataMember(entry.getValue().getHeader(), entry.getValue().getId())
+              .executeNonQueryPlan(entry.getKey());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    } else {
+      // forward the query to the group that should handle it
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Forward {} to a remote group of {}", entry.getKey(), entry.getValue().getHeader());
+      result = forwardPlan(entry.getKey(), entry.getValue());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    }
+    return result;
+  }
+
+  /**
+   * forward each sub-plan to its corresponding data group, if some groups goes wrong, the error
+   * messages from each group will be compacted into one string.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGroupMap) {

Review comment:
       same

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(

Review comment:
       It seems that this function will not be used?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(

Review comment:
       same

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)

Review comment:
       Then please add some comments for this parameters

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -32,25 +32,57 @@
 /** AddNodeLog records the operation of adding a node into this cluster. */
 public class AddNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node newNode;
 
-  public Node getNewNode() {
-    return newNode;
+  private long metaLogIndex;

Review comment:
       OK

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -165,14 +165,14 @@ public static Long querySingleSeries(
   }
 
   public static List<String> getNodeList(

Review comment:
       OK

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {
+    TSStatus result;
+    if (entry.getValue().contains(thisNode)) {
+      // the query should be handled by a group the local node is in, handle it with in the group
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Execute {} in a local group of {}", entry.getKey(), entry.getValue().getHeader());
+      result =
+          getLocalDataMember(entry.getValue().getHeader(), entry.getValue().getId())
+              .executeNonQueryPlan(entry.getKey());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    } else {
+      // forward the query to the group that should handle it
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Forward {} to a remote group of {}", entry.getKey(), entry.getValue().getHeader());
+      result = forwardPlan(entry.getKey(), entry.getValue());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    }
+    return result;
+  }
+
+  /**
+   * forward each sub-plan to its corresponding data group, if some groups goes wrong, the error
+   * messages from each group will be compacted into one string.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGroupMap) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    boolean allRedirect = true;
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      if (!tmpStatus.isSetRedirectNode()) {
+        allRedirect = false;
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        logger.error("Fail to send log {} to data group {}", entry.getKey(), entry.getValue());
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s]",
+                tmpStatus.getCode(), entry.getValue().getHeader(), tmpStatus.getMessage()));
+      }
+    }
+    TSStatus status;
+    if (errorCodePartitionGroups.isEmpty()) {
+      if (allRedirect) {
+        status = new TSStatus();
+        status.setCode(TSStatusCode.NEED_REDIRECTION.getStatusCode());
+      } else {
+        status = StatusUtils.OK;
+      }
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR, MSG_MULTIPLE_ERROR + errorCodePartitionGroups);
+    }
+    return status;
+  }
+
+  /**
+   * Forward a plan to the DataGroupMember of one node in the group. Only when all nodes time out,
+   * will a TIME_OUT be returned.
+   */
+  private TSStatus forwardPlan(PhysicalPlan plan, PartitionGroup group) {

Review comment:
       same

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       OK, I misunderstood




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644937843



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode());

Review comment:
       StatusUtils doesn't have a suitable static method. Although I can use `StatusUtils.getStatus(new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode()), endPoint)`, it will call one more deepClone(). I prefer not to do this only for save one line.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640715175



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
##########
@@ -146,11 +149,12 @@ private void parseRemoved(String[] split) {
     }
     DataGroupMember member = memberFactory.create(partitionGroup, thisNode);
     member.setReadOnly();
-    removedMemberMap.put(partitionGroup.getHeader(), member);
+    // TODO CORRECT
+    removedMemberMap.put(new RaftNode(partitionGroup.getHeader(), 0), member);

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.

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



[GitHub] [iotdb] fanhualta edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-853256335


   > > Please attach your design document or a link to it, I would recommend anyone read it before proceeding to the code review.
   > 
   > So, where is the cluster scalability design document? I only see a document about multi-raft in https://cwiki.apache.org/confluence/display/IOTDB/Multi-Raft-For-One-DataGroup+Design+Doc
   
   Please refer to https://cwiki.apache.org/confluence/display/IOTDB/Cluster+scalability+design+doc


-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-869149789


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [139 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='50.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [50.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.1%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.1% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865107395


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D.png' alt='D' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='54.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [54.0% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639865309



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
##########
@@ -70,4 +70,14 @@ void sendHeartbeatAsync(Node node) {
     // erase the sent partition table so it will not be sent in the next heartbeat
     request.unsetPartitionTableBytes();
   }
+
+  @Override
+  void startElection() {
+    //    if (localMetaMember.getThisNode().metaPort != 9003 &&
+    // localMetaMember.getThisNode().metaPort != 9005) {
+    //      return;
+    //    }
+    super.startElection();
+    localMetaMember.getAppendLogThreadPool().submit(() -> localMetaMember.processEmptyContentLog());

Review comment:
       A new raft leader needs to have at least one log in its term for committing logs with older terms. In the meta group, log frequency is very low. When the leader is changed whiling changing membership, it's necessary to process an empty log to make sure that cluster expansion operation can be carried out in time.
   
   I have added notes to explain in codes.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40800676/badge)](https://coveralls.io/builds/40800676)
   
   Coverage increased (+0.008%) to 67.894% when pulling **b2b6ec77d7928aca1d0d50df142257b29f8211c9 on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648487240



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -266,7 +273,7 @@ private TSStatus forwardPlan(List<PartitionGroup> partitionGroups, PhysicalPlan
         // the query should be handled by a group the local node is in, handle it with in the group
         status =
             metaGroupMember
-                .getLocalDataMember(partitionGroup.getHeader())
+                .getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId())

Review comment:
       It's misused. 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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644974459



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -435,32 +440,42 @@ public static TSStatus executeNonQuery(
   }
 
   public static ByteBuffer readFile(
-      AsyncDataClient client, String remotePath, long offset, int fetchSize)
+      AsyncDataClient client, String remotePath, long offset, int fetchSize, int raftId)

Review comment:
       I have removed raft from the following methods: 
   
   - readFile
   - removeHardLink
   
   Because these two methods are unassociated with raft group.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852261626


   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=coverage&view=list) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo.png' alt='No Duplication information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648466513



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
##########
@@ -144,6 +151,29 @@ private PartitionGroup routePlan(ShowChildPathsPlan plan) {
     throw new UnsupportedPlanException(plan);
   }
 
+  public Map<PhysicalPlan, PartitionGroup> splitAndRouteChangeMembershipLog(Log log) {
+    Map<PhysicalPlan, PartitionGroup> result = new HashMap<>();
+    LogPlan plan = new LogPlan(log.serialize());
+    List<Node> oldRing = new ArrayList<>(partitionTable.getAllNodes());
+    if (log instanceof AddNodeLog) {
+      oldRing.remove(((AddNodeLog) log).getNewNode());
+    } else if (log instanceof RemoveNodeLog) {
+      if (!oldRing.contains(((RemoveNodeLog) log).getRemovedNode())) {
+        oldRing.add(((RemoveNodeLog) log).getRemovedNode());
+        oldRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
+      }
+    }
+    for (PartitionGroup partitionGroup : partitionTable.calculateGlobalGroups(oldRing)) {
+      // It doesn't need to notify the data group which will be removed from cluster.
+      if (log instanceof RemoveNodeLog
+          && partitionGroup.getHeader().equals(((RemoveNodeLog) log).getRemovedNode())) {
+        continue;

Review comment:
       I have checked all errors like this and fixed them, it's due to the last commit.




-- 
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.

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



[GitHub] [iotdb] jt2594838 commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640236954



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
##########
@@ -70,4 +70,14 @@ void sendHeartbeatAsync(Node node) {
     // erase the sent partition table so it will not be sent in the next heartbeat
     request.unsetPartitionTableBytes();
   }
+
+  @Override
+  void startElection() {
+    //    if (localMetaMember.getThisNode().metaPort != 9003 &&
+    // localMetaMember.getThisNode().metaPort != 9005) {
+    //      return;
+    //    }
+    super.startElection();
+    localMetaMember.getAppendLogThreadPool().submit(() -> localMetaMember.processEmptyContentLog());

Review comment:
       Understood, but this is also reachable for non-leader nodes, better to put it where the election is won.




-- 
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.

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



[GitHub] [iotdb] jt2594838 commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-850268094


   Please consider adding more tests, current tests are very limited.


-- 
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.

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



[GitHub] [iotdb] mychaow commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-850261253


   > Please attach your design document or a link to it, I would recommend anyone read it before proceeding to the code review.
   
   So, where is the cluster scalability design document? I only see a document about multi-raft in https://cwiki.apache.org/confluence/display/IOTDB/Multi-Raft-For-One-DataGroup+Design+Doc


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639858683



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
##########
@@ -82,16 +85,15 @@ private void resetAggregateResults() {
                 .getAsyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
         aggrBuffers =
             SyncClientAdaptor.getGroupByResult(
-                client, header, executorId, curStartTime, curEndTime);
+                client, header, raftId, executorId, curStartTime, curEndTime);
       } else {
-        try (SyncDataClient syncDataClient =
+        SyncDataClient syncDataClient =
             metaGroupMember
                 .getClientProvider()
-                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-
-          aggrBuffers =
-              syncDataClient.getGroupByResult(header, executorId, curStartTime, curEndTime);
-        }
+                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
+        aggrBuffers =
+            syncDataClient.getGroupByResult(header, raftId, executorId, curStartTime, curEndTime);
+        ClientUtils.putBackSyncClient(syncDataClient);

Review comment:
       Fixed: use try() to wrap it which is same with master branch.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40824795/badge)](https://coveralls.io/builds/40824795)
   
   Coverage decreased (-0.01%) to 67.873% when pulling **56f9ddad01bde745a035b60d0c66f96d6c203861 on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644869654



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -31,17 +31,49 @@
 
 public class RemoveNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node removedNode;
 
+  private long metaLogIndex;

Review comment:
       The answer is the same as above.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640748780



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1602,38 +1971,119 @@ private long processRemoveNodeLocally(Node node) throws LogExecutionException {
       return Response.RESPONSE_REJECT;
     }
 
+    RemoveNodeLog removeNodeLog = new RemoveNodeLog();
     // node removal must be serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable(thisNode);
+      table.deserialize(partitionTable.serialize());

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.

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



[GitHub] [iotdb] jt2594838 commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r632349537



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -499,18 +515,26 @@ public static ByteBuffer last(
             context.getQueryId(),
             deviceMeasurements,
             header,
+            raftId,
             client.getNode());
 
     client.last(request, handler);
     return handler.getResult(RaftServer.getReadOperationTimeoutMS());
   }
 
-  public static boolean onSnapshotApplied(AsyncDataClient client, Node header, List<Integer> slots)
+  public static boolean onSnapshotApplied(
+      AsyncDataClient client, Node header, int raftId, List<Integer> slots)
       throws TException, InterruptedException {
-    AtomicReference<Boolean> result = new AtomicReference<>(false);
+    AtomicReference<Boolean> result = new AtomicReference<>();
     GenericHandler<Boolean> handler = new GenericHandler<>(client.getNode(), result);
 
-    client.onSnapshotApplied(header, slots, handler);
-    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+    client.onSnapshotApplied(header, raftId, slots, handler);
+
+    synchronized (result) {
+      if (result.get() == null) {
+        result.wait();
+      }
+    }

Review comment:
       If an error occurs before the synchronization block, the result will never be notified.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,46 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log)
+      throws ChangeMembershipException, InterruptedException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }

Review comment:
       More than one node may consider themselves a leader, so is it safe if the log is sent more than once?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
##########
@@ -146,11 +149,12 @@ private void parseRemoved(String[] split) {
     }
     DataGroupMember member = memberFactory.create(partitionGroup, thisNode);
     member.setReadOnly();
-    removedMemberMap.put(partitionGroup.getHeader(), member);
+    // TODO CORRECT
+    removedMemberMap.put(new RaftNode(partitionGroup.getHeader(), 0), member);

Review comment:
       Maybe you can add the raftId into the file during serialization and deserialize them when start-up.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
##########
@@ -919,10 +948,18 @@ protected boolean waitUntilCatchUp(CheckConsistency checkConsistency)
    * @param leaderCommitId leader commit id
    * @return true if leaderCommitId <= localAppliedId
    */
-  private boolean syncLocalApply(long leaderCommitId) {
+  public boolean syncLocalApply(long leaderCommitId) {
     long startTime = System.currentTimeMillis();
     long waitedTime = 0;
     long localAppliedId = 0;
+
+    // If the leader and follower logs differ too much, local query is not allowed
+    if (leaderCommitId - logManager.getMaxHaveAppliedCommitIndex() > 1000) {
+      logger.info(
+          "{}: The raft log of this member is too backward to provide service directly.", name);
+      return false;
+    }

Review comment:
       This should be configurable.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -810,11 +810,11 @@ public void updatePlanIndexes(TsFileResource another) {
   }
 
   public boolean isPlanIndexOverlap(TsFileResource another) {
-    return another.maxPlanIndex >= this.minPlanIndex && another.minPlanIndex <= this.maxPlanIndex;
+    return another.maxPlanIndex > this.minPlanIndex && another.minPlanIndex < this.maxPlanIndex;
   }
 
   public boolean isPlanRangeCovers(TsFileResource another) {
-    return this.minPlanIndex <= another.minPlanIndex && another.maxPlanIndex <= this.maxPlanIndex;
+    return this.minPlanIndex < another.minPlanIndex && another.maxPlanIndex < this.maxPlanIndex;
   }

Review comment:
       Is this really necessary? This may cause many more redundant files to be loaded if two nodes generate similar files.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -2605,39 +2605,61 @@ private boolean loadTsFileByType(
       targetFile.getParentFile().mkdirs();
     }
     try {
-      FileUtils.moveFile(syncedTsFile, targetFile);
+      FileUtils.moveFile(tsFileToLoad, targetFile);
     } catch (IOException e) {
       logger.error(
           "File renaming failed when loading tsfile. Origin: {}, Target: {}",
-          syncedTsFile.getAbsolutePath(),
+          tsFileToLoad.getAbsolutePath(),
           targetFile.getAbsolutePath(),
           e);
       throw new LoadFileException(
           String.format(
               "File renaming failed when loading tsfile. Origin: %s, Target: %s, because %s",
-              syncedTsFile.getAbsolutePath(), targetFile.getAbsolutePath(), e.getMessage()));
+              tsFileToLoad.getAbsolutePath(), targetFile.getAbsolutePath(), e.getMessage()));
     }
 
-    File syncedResourceFile =
-        fsFactory.getFile(syncedTsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    File resourceFileToLoad =
+        fsFactory.getFile(tsFileToLoad.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
     File targetResourceFile =
         fsFactory.getFile(targetFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
     try {
-      FileUtils.moveFile(syncedResourceFile, targetResourceFile);
+      FileUtils.moveFile(resourceFileToLoad, targetResourceFile);
     } catch (IOException e) {
       logger.error(
           "File renaming failed when loading .resource file. Origin: {}, Target: {}",
-          syncedResourceFile.getAbsolutePath(),
+          resourceFileToLoad.getAbsolutePath(),
           targetResourceFile.getAbsolutePath(),
           e);
       throw new LoadFileException(
           String.format(
               "File renaming failed when loading .resource file. Origin: %s, Target: %s, because %s",
-              syncedResourceFile.getAbsolutePath(),
+              resourceFileToLoad.getAbsolutePath(),
               targetResourceFile.getAbsolutePath(),
               e.getMessage()));
     }
 
+    File modFileToLoad =
+        fsFactory.getFile(tsFileToLoad.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    if (modFileToLoad.exists()) {
+      File targetModFile =
+          fsFactory.getFile(targetFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);

Review comment:
       The variable is `modFileToLoad` while the suffix is RESOURCE_SUFFIX.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -79,19 +84,39 @@
   List<PartitionGroup> getLocalGroups();
 
   /**
-   * @param header
+   * @param raftNode
    * @return the partition group starting from the header.
    */
-  PartitionGroup getHeaderGroup(Node header);
+  PartitionGroup getHeaderGroup(RaftNode raftNode);
+
+  PartitionGroup getHeaderGroup(Node node);
 
   ByteBuffer serialize();
 
-  void deserialize(ByteBuffer buffer);
+  /**
+   * Deserialize partition table and check whether the partition table in byte buffer is valid
+   *
+   * @param buffer
+   * @return true if the partition table is valid
+   */
+  boolean deserialize(ByteBuffer buffer);
 
   List<Node> getAllNodes();
 
   List<PartitionGroup> getGlobalGroups();
 
+  List<PartitionGroup> calculateGlobalGroups(List<Node> nodeRing);
+
+  /**
+   * Judge whether the data of slot is held by node
+   *
+   * @param node target node
+   */
+  boolean judgeHoldSlot(Node node, int slot);

Review comment:
       `Slot` should be avoided in the interface definition, as PartitionTable may not bind to implementation with slots.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
##########
@@ -173,23 +181,35 @@ private boolean pullSnapshot(int nodeIndex) throws InterruptedException, TExcept
 
   @Override
   public Void call() {
-    persistTask();
     request = new PullSnapshotRequest();
     request.setHeader(descriptor.getPreviousHolders().getHeader());
+    request.setRaftId(descriptor.getPreviousHolders().getId());
     request.setRequiredSlots(descriptor.getSlots());
     request.setRequireReadOnly(descriptor.isRequireReadOnly());
 
+    logger.info("{}: data migration starts.", newMember.getName());
     boolean finished = false;
-    int nodeIndex = -1;
+    int nodeIndex = ((PartitionGroup) newMember.getAllNodes()).indexOf(newMember.getThisNode()) - 1;
     while (!finished) {
       try {
         // sequentially pick up a node that may have this slot
         nodeIndex = (nodeIndex + 1) % descriptor.getPreviousHolders().size();
+        long startTime = System.currentTimeMillis();
         finished = pullSnapshot(nodeIndex);
         if (!finished) {
+          if (logger.isDebugEnabled()) {
+            logger.debug(
+                "Cannot pull slot {} from {}, retry",
+                descriptor.getSlots(),
+                descriptor.getPreviousHolders().get(nodeIndex));
+          }
           Thread.sleep(
               ClusterDescriptor.getInstance().getConfig().getPullSnapshotRetryIntervalMs());
         }
+        logger.debug(
+            "{}: Data migration ends, cost {}ms",
+            newMember,
+            (System.currentTimeMillis() - startTime));

Review comment:
       This log should be put into an else block.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
##########
@@ -175,11 +219,21 @@ public PartitionGroup getHeaderGroup(Node node) {
     return ret;
   }
 
+  @Override
+  public PartitionGroup getHeaderGroup(RaftNode raftNode) {
+    return getHeaderGroup(raftNode, this.nodeRing);
+  }
+
+  @Override
+  public PartitionGroup getHeaderGroup(Node node) {
+    return getHeaderGroup(new RaftNode(node, 0));
+  }

Review comment:
       Is this interface still useful anymore?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
##########
@@ -96,19 +96,21 @@ public boolean hasNextDataClient(boolean byTimestamp, long timestamp) {
         Long newReaderId = getReaderId(node, byTimestamp, timestamp);
         if (newReaderId != null) {
           logger.debug("get a readerId {} for {} from {}", newReaderId, request.path, node);
-          if (newReaderId != -1) {
+          if (newReaderId >= 0) {
             // register the node so the remote resources can be released
-            context.registerRemoteNode(node, partitionGroup.getHeader());
+            context.registerRemoteNode(node, partitionGroup.getHeader(), partitionGroup.getId());
             this.readerId = newReaderId;
             this.curSource = node;
             this.curPos = nextNodePos;
             return true;
-          } else {
+          } else if (newReaderId == -1) {
             // the id being -1 means there is no satisfying data on the remote node, create an
             // empty reader to reduce further communication
             this.isNoClient = true;
             this.isNoData = true;
             return false;
+          } else {
+            logger.debug("change other client for better query performance.");

Review comment:
       Could you please further explain this?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus();
+        status.setCode(TSStatusCode.NEED_REDIRECTION.getStatusCode());
+      } else {
+        status = StatusUtils.OK;
       }

Review comment:
       Where is the endPoint provided now?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
##########
@@ -70,4 +70,14 @@ void sendHeartbeatAsync(Node node) {
     // erase the sent partition table so it will not be sent in the next heartbeat
     request.unsetPartitionTableBytes();
   }
+
+  @Override
+  void startElection() {
+    //    if (localMetaMember.getThisNode().metaPort != 9003 &&
+    // localMetaMember.getThisNode().metaPort != 9005) {
+    //      return;
+    //    }
+    super.startElection();
+    localMetaMember.getAppendLogThreadPool().submit(() -> localMetaMember.processEmptyContentLog());

Review comment:
       What is this for?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/utils/nodetool/ClusterMonitor.java
##########
@@ -66,9 +77,67 @@ public void start() throws StartupException {
   }
 
   @Override
+  public List<Pair<Node, NodeCharacter>> getMetaGroup() {
+    MetaGroupMember metaMember = getMetaGroupMember();
+    if (metaMember == null || metaMember.getPartitionTable() == null) {
+      return null;
+    }
+    List<Pair<Node, NodeCharacter>> res = new ArrayList<>();
+    for (Node node : metaMember.getPartitionTable().getAllNodes()) {
+      if (node.equals(metaMember.getLeader())) {
+        res.add(new Pair<>(node, NodeCharacter.LEADER));
+      } else {
+        res.add(new Pair<>(node, NodeCharacter.FOLLOWER));
+      }
+    }
+    return res;

Review comment:
       I would recommend you copy the current leader and nodes outside of the loop because they may change during the iteration so the caller may get two or more leaders.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -869,39 +929,116 @@ private boolean processAddNodeLocally(
       return true;
     }
 
+    AddNodeLog addNodeLog = new AddNodeLog();
     // node adding is serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      AddNodeLog addNodeLog = new AddNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable(thisNode);
+      table.deserialize(partitionTable.serialize());
+      table.addNode(newNode);
+      ((SlotPartitionTable) table).setLastMetaLogIndex(logManager.getLastLogIndex() + 1);
+
+      addNodeLog.setPartitionTable(table.serialize());
       addNodeLog.setCurrLogTerm(getTerm().get());
       addNodeLog.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+      addNodeLog.setMetaLogIndex(logManager.getLastLogIndex() + 1);
 
-      addNodeLog.setNewNode(node);
+      addNodeLog.setNewNode(newNode);
 
       logManager.append(addNodeLog);
+    }
 
-      int retryTime = 1;
-      while (true) {
-        logger.info("Send the join request of {} to other nodes, retry time: {}", node, retryTime);
-        AppendLogResult result = sendLogToAllGroups(addNodeLog);
-        switch (result) {
-          case OK:
-            logger.info("Join request of {} is accepted", node);
-            commitLog(addNodeLog);
-
-            synchronized (partitionTable) {
-              response.setPartitionTableBytes(partitionTable.serialize());
-            }
-            response.setRespNum((int) Response.RESPONSE_AGREE);
-            logger.info("Sending join response of {}", node);
-            return true;
-          case TIME_OUT:
-            logger.info("Join request of {} timed out", node);
-            retryTime++;
-            continue;
-          case LEADERSHIP_STALE:
-          default:
-            return false;
-        }
+    int retryTime = 0;
+    while (true) {
+      logger.info(
+          "{}: Send the join request of {} to other nodes, retry time: {}",
+          name,
+          newNode,
+          retryTime);
+      AppendLogResult result = sendLogToFollowers(addNodeLog);
+      switch (result) {
+        case OK:
+          commitLog(addNodeLog);
+          logger.info("{}: Join request of {} is accepted", name, newNode);
+
+          synchronized (partitionTable) {
+            response.setPartitionTableBytes(partitionTable.serialize());
+          }
+          response.setRespNum((int) Response.RESPONSE_AGREE);
+          logger.info("{}: Sending join response of {}", name, newNode);
+          return true;
+        case TIME_OUT:
+          logger.debug("{}: log {} timed out, retrying...", name, addNodeLog);
+          try {
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+          logger.info("{}: Join request of {} timed out", name, newNode);
+          retryTime++;
+          break;
+        case LEADERSHIP_STALE:
+        default:
+          return false;
+      }
+    }
+  }
+
+  /** Check if there has data migration due to previous change membership operation. */
+  private boolean waitDataMigrationEnd() throws InterruptedException, CheckConsistencyException {
+    // try 5 time
+    int retryTime = 0;
+    while (true) {
+      Map<PartitionGroup, Integer> res = collectAllPartitionMigrationStatus();
+      if (res != null && res.isEmpty()) {
+        return true;
+      }
+      if (++retryTime == 5) {
+        break;
+      }
+      Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+    }
+    return false;
+  }
+
+  /** Process empty log for leader to commit all previous log. */
+  public void processEmptyContentLog() {
+    if (character != NodeCharacter.LEADER) {
+      return;
+    }
+
+    Log log = new EmptyContentLog();
+    log.setCurrLogTerm(getTerm().get());
+    log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+
+    synchronized (logManager) {
+      logManager.append(log);
+    }

Review comment:
       The index assignment is outside of the synchronized block, so is it possible that several logs have conflict indices?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
##########
@@ -82,16 +85,15 @@ private void resetAggregateResults() {
                 .getAsyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
         aggrBuffers =
             SyncClientAdaptor.getGroupByResult(
-                client, header, executorId, curStartTime, curEndTime);
+                client, header, raftId, executorId, curStartTime, curEndTime);
       } else {
-        try (SyncDataClient syncDataClient =
+        SyncDataClient syncDataClient =
             metaGroupMember
                 .getClientProvider()
-                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-
-          aggrBuffers =
-              syncDataClient.getGroupByResult(header, executorId, curStartTime, curEndTime);
-        }
+                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
+        aggrBuffers =
+            syncDataClient.getGroupByResult(header, raftId, executorId, curStartTime, curEndTime);
+        ClientUtils.putBackSyncClient(syncDataClient);

Review comment:
       `putBackSyncClient` should be in a finally block to avoid resource leak.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -933,29 +1011,83 @@ public IBatchReader getSeriesBatchReader(
       Filter valueFilter,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots,
+      boolean syncLeader)
       throws StorageEngineException, QueryProcessException, IOException {
-    // pull the newest data
-    try {
-      dataGroupMember.syncLeaderWithConsistencyCheck(false);
-    } catch (CheckConsistencyException e) {
-      throw new StorageEngineException(e);
+    if (syncLeader) {
+      // pull the newest data
+      try {
+        dataGroupMember.syncLeaderWithConsistencyCheck(false);
+      } catch (CheckConsistencyException e) {
+        throw new StorageEngineException(e);
+      }
     }
 
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            timeFilter,
-            valueFilter,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
-    if (seriesReader.isEmpty()) {
-      return null;
+    // find the groups that should be queried due to data migration.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
+
+    // If requiredSlots is not null, it means that this data group is the previous holder of
+    // required slots, which is no need to merge other resource,
+    if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+      // merge remote reader and local reader
+      ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+      // add local reader
+      IPointReader seriesPointReader =
+          getSeriesPointReader(
+              path,
+              allSensors,
+              dataType,
+              timeFilter,
+              valueFilter,
+              context,
+              dataGroupMember,
+              ascending,
+              requiredSlots);
+      mergeReader.addReader(seriesPointReader, 0);

Review comment:
       Maybe the local reader should have a higher priority or be placed backwards since data in the new hold is newer than the old holder.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1501,11 +1814,20 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
       Thread.currentThread().interrupt();
       logger.warn("Cannot get the status of all nodes", e);
     }
+
+    for (Node node : partitionTable.getAllNodes()) {
+      nodeStatus.putIfAbsent(node, 2);
+    }
+    for (Node node : allNodes) {
+      if (!partitionTable.getAllNodes().contains(node)) {
+        nodeStatus.put(node, 3);
+      }
+    }

Review comment:
       Better to replace these numbers with meaningful constants or enums.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);

Review comment:
       Meta followers may apply this meta log before they apply the corresponding data log, is it safe?

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -254,7 +254,7 @@ public void testInstallMultiple()
       List<TsFileResource> loadedFiles = processor.getSequenceFileTreeSet();
       assertEquals(10, loadedFiles.size());
       for (int i = 0; i < 9; i++) {
-        assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
+        assertEquals(-1, loadedFiles.get(i).getMaxPlanIndex());

Review comment:
       This case tests snapshot catch-up, so the plan indices in the files should be reserved. Please check what went wrong.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1602,38 +1971,119 @@ private long processRemoveNodeLocally(Node node) throws LogExecutionException {
       return Response.RESPONSE_REJECT;
     }
 
+    RemoveNodeLog removeNodeLog = new RemoveNodeLog();
     // node removal must be serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable(thisNode);
+      table.deserialize(partitionTable.serialize());

Review comment:
       It would be better to replace this with a copy constructor.

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -301,6 +301,6 @@ public void testInstallPartial()
     for (int i = 0; i < 9; i++) {
       assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
     }
-    assertEquals(0, processor.getUnSequenceFileList().size());
+    assertEquals(1, processor.getUnSequenceFileList().size());

Review comment:
       Why should there be an unseq file?

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/LogParserTest.java
##########
@@ -98,4 +104,14 @@ public void testEmptyContentLog() throws UnknownLogTypeException {
     Log serialized = logParser.parse(byteBuffer);
     assertEquals(log, serialized);
   }
+
+  @Test
+  public void testLogPlan() throws IOException, IllegalPathException, UnknownLogTypeException {
+    AddNodeLog log = new AddNodeLog(TestUtils.seralizePartitionTable, TestUtils.getNode(0));
+    log.setMetaLogIndex(1);
+    LogPlan logPlan = new LogPlan(log.serialize());
+    ByteBuffer buffer = ByteBuffer.wrap(PlanSerializer.getInstance().serialize(logPlan));
+    PhysicalPlan plan = PhysicalPlan.Factory.create(buffer);
+    LogParser.getINSTANCE().parse(((LogPlan) plan).getLog());
+  }

Review comment:
       Add an assertion to the test.

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java
##########
@@ -298,7 +299,7 @@ private void testNormal(boolean requiresReadOnly)
             loadedFiles.get(i).getMaxPlanIndex(),
             loadedFiles.get(i).getTsFile().getAbsolutePath());
       }
-      assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
+      assertEquals(-1, loadedFiles.get(i).getMaxPlanIndex());

Review comment:
       This case is not for data migration so the plan indices should not be changed.

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java
##########
@@ -1175,16 +1210,16 @@ public void testRemoteAddNode() {
       assertTrue(response.getCheckStatusResponse().isClusterNameEquals());
 
       // cannot add a node due to network failure
-      dummyResponse.set(Response.RESPONSE_NO_CONNECTION);
-      testMetaMember.setCharacter(LEADER);
-      result.set(null);
-      testMetaMember.setPartitionTable(partitionTable);
-      new Thread(
-              () -> {
-                await().atLeast(200, TimeUnit.MILLISECONDS);
-                dummyResponse.set(Response.RESPONSE_AGREE);
-              })
-          .start();
+      //      dummyResponse.set(Response.RESPONSE_NO_CONNECTION);
+      //      testMetaMember.setCharacter(LEADER);
+      //      result.set(null);
+      //      testMetaMember.setPartitionTable(partitionTable);
+      //      new Thread(
+      //              () -> {
+      //                await().atLeast(200, TimeUnit.MILLISECONDS);
+      //                dummyResponse.set(Response.RESPONSE_AGREE);
+      //              })
+      //          .start();

Review comment:
       Remove or resume this.




-- 
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.

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



[GitHub] [iotdb] coveralls commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40209317/badge)](https://coveralls.io/builds/40209317)
   
   Coverage increased (+0.02%) to 67.924% when pulling **57cdff53c4b77bdbf2289c39fa478f605a6f28a9 on cluster_scalability** into **a99bf9c78eb02f2258f442ae7af725f3fff23ad3 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644911199



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -73,6 +92,10 @@ public void waitSlot(int slotId) {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > 1000) {

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640790678



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -301,6 +301,6 @@ public void testInstallPartial()
     for (int i = 0; i < 9; i++) {
       assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
     }
-    assertEquals(0, processor.getUnSequenceFileList().size());
+    assertEquals(1, processor.getUnSequenceFileList().size());

Review comment:
       Because the logic of judging whether the file exists according to the plan index has changed, the file is not considered to exist when the index plan is equal to. So in the test `testInstallPartial`,  there will be one file to be judged as an unseq file.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40764743/badge)](https://coveralls.io/builds/40764743)
   
   Coverage increased (+0.02%) to 67.91% when pulling **f815fccad2cfa656e577cd16e8b301cd48a224f6 on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648778807



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);

Review comment:
       > If a slot is owned by B datagroup, it will be moved to A datagroup in the new partition table. If A datagroup performs this step first, it will start a new datagroupmember to receive new writes. However, B datagroup still has a follower that has not been implemented to this step. If a write request is sent to it, it will route to the leader of the raft group, and the leader will execute it directly. When A datagroup pulls a slot later, it may lose some data?
   
   Hi, PTAL.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644124436



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
##########
@@ -677,6 +677,9 @@ public boolean matchTerm(long term, long index) {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
+      if (entry.isApplied()) {

Review comment:
       For add/remove logs in data groups, this log will be applied immediately when it is appended to the raft log.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644926346



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -337,7 +362,8 @@ private AbstractMultPointReader getMultSeriesReader(
           deviceMeasurements,
           partitionGroup,
           context,
-          ascending);
+          ascending,
+          requiredSlots);

Review comment:
       I have checked the logic and remove useless parameters.




-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-854297217


   Please add some user guide in `https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster` and add some examples for add/remove node operation


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639867296



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -869,39 +929,116 @@ private boolean processAddNodeLocally(
       return true;
     }
 
+    AddNodeLog addNodeLog = new AddNodeLog();
     // node adding is serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      AddNodeLog addNodeLog = new AddNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable(thisNode);
+      table.deserialize(partitionTable.serialize());
+      table.addNode(newNode);
+      ((SlotPartitionTable) table).setLastMetaLogIndex(logManager.getLastLogIndex() + 1);
+
+      addNodeLog.setPartitionTable(table.serialize());
       addNodeLog.setCurrLogTerm(getTerm().get());
       addNodeLog.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+      addNodeLog.setMetaLogIndex(logManager.getLastLogIndex() + 1);
 
-      addNodeLog.setNewNode(node);
+      addNodeLog.setNewNode(newNode);
 
       logManager.append(addNodeLog);
+    }
 
-      int retryTime = 1;
-      while (true) {
-        logger.info("Send the join request of {} to other nodes, retry time: {}", node, retryTime);
-        AppendLogResult result = sendLogToAllGroups(addNodeLog);
-        switch (result) {
-          case OK:
-            logger.info("Join request of {} is accepted", node);
-            commitLog(addNodeLog);
-
-            synchronized (partitionTable) {
-              response.setPartitionTableBytes(partitionTable.serialize());
-            }
-            response.setRespNum((int) Response.RESPONSE_AGREE);
-            logger.info("Sending join response of {}", node);
-            return true;
-          case TIME_OUT:
-            logger.info("Join request of {} timed out", node);
-            retryTime++;
-            continue;
-          case LEADERSHIP_STALE:
-          default:
-            return false;
-        }
+    int retryTime = 0;
+    while (true) {
+      logger.info(
+          "{}: Send the join request of {} to other nodes, retry time: {}",
+          name,
+          newNode,
+          retryTime);
+      AppendLogResult result = sendLogToFollowers(addNodeLog);
+      switch (result) {
+        case OK:
+          commitLog(addNodeLog);
+          logger.info("{}: Join request of {} is accepted", name, newNode);
+
+          synchronized (partitionTable) {
+            response.setPartitionTableBytes(partitionTable.serialize());
+          }
+          response.setRespNum((int) Response.RESPONSE_AGREE);
+          logger.info("{}: Sending join response of {}", name, newNode);
+          return true;
+        case TIME_OUT:
+          logger.debug("{}: log {} timed out, retrying...", name, addNodeLog);
+          try {
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+          logger.info("{}: Join request of {} timed out", name, newNode);
+          retryTime++;
+          break;
+        case LEADERSHIP_STALE:
+        default:
+          return false;
+      }
+    }
+  }
+
+  /** Check if there has data migration due to previous change membership operation. */
+  private boolean waitDataMigrationEnd() throws InterruptedException, CheckConsistencyException {
+    // try 5 time
+    int retryTime = 0;
+    while (true) {
+      Map<PartitionGroup, Integer> res = collectAllPartitionMigrationStatus();
+      if (res != null && res.isEmpty()) {
+        return true;
+      }
+      if (++retryTime == 5) {
+        break;
+      }
+      Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+    }
+    return false;
+  }
+
+  /** Process empty log for leader to commit all previous log. */
+  public void processEmptyContentLog() {
+    if (character != NodeCharacter.LEADER) {
+      return;
+    }
+
+    Log log = new EmptyContentLog();
+    log.setCurrLogTerm(getTerm().get());
+    log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+
+    synchronized (logManager) {
+      logManager.append(log);
+    }

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644913894



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -1023,30 +1157,85 @@ public IReaderByTimestamp getReaderByTimestamp(
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                requiredSlots);

Review comment:
       Yes. I have specified it as null.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640727137



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1501,11 +1814,20 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
       Thread.currentThread().interrupt();
       logger.warn("Cannot get the status of all nodes", e);
     }
+
+    for (Node node : partitionTable.getAllNodes()) {
+      nodeStatus.putIfAbsent(node, 2);
+    }
+    for (Node node : allNodes) {
+      if (!partitionTable.getAllNodes().contains(node)) {
+        nodeStatus.put(node, 3);
+      }
+    }

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639892382



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
##########
@@ -175,11 +219,21 @@ public PartitionGroup getHeaderGroup(Node node) {
     return ret;
   }
 
+  @Override
+  public PartitionGroup getHeaderGroup(RaftNode raftNode) {
+    return getHeaderGroup(raftNode, this.nodeRing);
+  }
+
+  @Override
+  public PartitionGroup getHeaderGroup(Node node) {
+    return getHeaderGroup(new RaftNode(node, 0));
+  }

Review comment:
       Just for compatibility. I have removed the interface now.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644880960



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)

Review comment:
       Yes, but the parameter is necessary. With this parameter, this method is more complete.




-- 
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.

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r646250496



##########
File path: cluster/src/assembly/resources/sbin/remove-node.sh
##########
@@ -0,0 +1,88 @@
+#!/bin/bash
+#
+# 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.
+#
+
+
+echo ---------------------
+echo "Starting to remove a node(Cluster Mode)"
+echo ---------------------
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="`dirname "$0"`/.."
+fi
+
+IOTDB_CONF=${IOTDB_HOME}/conf
+
+is_conf_path=false
+for arg do
+    shift
+    if [ "$arg" == "-c" ]; then
+        is_conf_path=true
+        continue

Review comment:
       The same as above.

##########
File path: cluster/src/assembly/resources/sbin/remove-node.bat
##########
@@ -0,0 +1,117 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+echo ````````````````````````
+echo Starting to remove a node (Cluster Mode)
+echo ````````````````````````
+
+PATH %PATH%;%JAVA_HOME%\bin\
+set "FULL_VERSION="
+set "MAJOR_VERSION="
+set "MINOR_VERSION="
+
+
+for /f tokens^=2-5^ delims^=.-_+^" %%j in ('java -fullversion 2^>^&1') do (
+	set "FULL_VERSION=%%j-%%k-%%l-%%m"
+	IF "%%j" == "1" (
+	    set "MAJOR_VERSION=%%k"
+	    set "MINOR_VERSION=%%l"
+	) else (
+	    set "MAJOR_VERSION=%%j"
+	    set "MINOR_VERSION=%%k"
+	)
+)
+
+set JAVA_VERSION=%MAJOR_VERSION%
+
+IF NOT %JAVA_VERSION% == 8 (
+	IF NOT %JAVA_VERSION% == 11 (
+		echo IoTDB only supports jdk8 or jdk11, please check your java version.
+		goto finally
+	)
+)
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
+popd
+
+set IOTDB_CONF=%IOTDB_HOME%\conf
+set IOTDB_LOGS=%IOTDB_HOME%\logs
+
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-r
+set is_conf_path=false
+for %%i in (%*) do (
+	IF "%%i" == "-c" (
+		set is_conf_path=true
+	) ELSE IF "!is_conf_path!" == "true" (

Review comment:
       The `-c` params is not supported in the cluster mode, maybe we  should remove here.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
##########
@@ -144,6 +151,29 @@ private PartitionGroup routePlan(ShowChildPathsPlan plan) {
     throw new UnsupportedPlanException(plan);
   }
 
+  public Map<PhysicalPlan, PartitionGroup> splitAndRouteChangeMembershipLog(Log log) {
+    Map<PhysicalPlan, PartitionGroup> result = new HashMap<>();
+    LogPlan plan = new LogPlan(log.serialize());
+    List<Node> oldRing = new ArrayList<>(partitionTable.getAllNodes());
+    if (log instanceof AddNodeLog) {
+      oldRing.remove(((AddNodeLog) log).getNewNode());
+    } else if (log instanceof RemoveNodeLog) {
+      if (!oldRing.contains(((RemoveNodeLog) log).getRemovedNode())) {
+        oldRing.add(((RemoveNodeLog) log).getRemovedNode());
+        oldRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
+      }
+    }
+    for (PartitionGroup partitionGroup : partitionTable.calculateGlobalGroups(oldRing)) {
+      // It doesn't need to notify the data group which will be removed from cluster.
+      if (log instanceof RemoveNodeLog
+          && partitionGroup.getHeader().equals(((RemoveNodeLog) log).getRemovedNode())) {
+        continue;

Review comment:
       `partitionGroup.getHeader()` is one instance of `RaftNode`, while `(RemoveNodeLog) log).getRemovedNode())` return the `Node` instance,  the two Object can not compare, Maybe we should sort out the relationship among `ClusterNode`, `RaftNode` and `Node` in the cluster module.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1759,17 +1951,17 @@ public void setAllNodes(List<Node> allNodes) {
    * @param request the toString() of this parameter should explain what the request is and it is
    *     only used in logs for tracing
    */
-  public DataGroupMember getLocalDataMember(Node header, Object request) {
+  public DataGroupMember getLocalDataMember(RaftNode header, Object request) {
     return dataClusterServer.getDataMember(header, null, request);
   }
 

Review comment:
       The method signature says the `request` is the toString() of this parameter should explain what the request is and it is only used in logs for tracing, however the caller user `raftId` like following, is it being misused?
   `metaGroupMember.getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId());`

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -304,6 +311,74 @@ private TSStatus forwardPlan(List<PartitionGroup> partitionGroups, PhysicalPlan
     return status;
   }
 
+  public void sendLogToAllDataGroups(Log log) throws ChangeMembershipException {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: start", log);
+    }
+
+    Map<PhysicalPlan, PartitionGroup> planGroupMap = router.splitAndRouteChangeMembershipLog(log);
+    List<String> errorCodePartitionGroups = new CopyOnWriteArrayList<>();
+    CountDownLatch counter = new CountDownLatch(planGroupMap.size());
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      metaGroupMember
+          .getAppendLogThreadPool()
+          .submit(() -> forwardChangeMembershipPlan(log, entry, errorCodePartitionGroups, counter));
+    }
+    try {
+      counter.await();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new ChangeMembershipException(
+          String.format("Can not wait all data groups to apply %s", log));
+    }
+    if (!errorCodePartitionGroups.isEmpty()) {
+      throw new ChangeMembershipException(
+          String.format("Apply %s failed with status {%s}", log, errorCodePartitionGroups));
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: end", log);
+    }
+  }
+
+  private void forwardChangeMembershipPlan(
+      Log log,
+      Map.Entry<PhysicalPlan, PartitionGroup> entry,
+      List<String> errorCodePartitionGroups,
+      CountDownLatch counter) {
+    int retryTime = 0;
+    try {
+      while (true) {
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "Send change membership log {} to data group {}, retry time: {}",
+              log,
+              entry.getValue(),
+              retryTime);
+        }
+        try {
+          TSStatus status = forwardToSingleGroup(entry);
+          if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+            if (logger.isDebugEnabled()) {
+              logger.debug(
+                  "Success to send change membership log {} to data group {}",
+                  log,
+                  entry.getValue());
+            }
+            return;
+          }
+          Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          errorCodePartitionGroups.add(e.getMessage());
+          return;
+        }
+        retryTime++;
+      }
+    } finally {
+      counter.countDown();
+    }
+  }

Review comment:
       Add timeout logic here? if this `DataGroup` has some problems, then the whole operation will be blocked, and thus the upper called layer will be blocked, and the operation of the `metaGroupMember` will be blocked.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -266,7 +273,7 @@ private TSStatus forwardPlan(List<PartitionGroup> partitionGroups, PhysicalPlan
         // the query should be handled by a group the local node is in, handle it with in the group
         status =
             metaGroupMember
-                .getLocalDataMember(partitionGroup.getHeader())
+                .getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId())

Review comment:
       Please see the `public DataGroupMember getLocalDataMember(RaftNode header, Object request)` signature  and make sure is it being misused? maybe we can  design one new method called `getLocalDataMember(RaftNode header, int raftId)`?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855427551


   > My review has almost been completed. Next, please do two things:
   > 
   > * Fix CI
   > * Add some user guide in https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster &&https://github.com/apache/iotdb/tree/master/docs/UserGuide/Cluster and add some examples for add/remove node operation
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639868329



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);

Review comment:
       It's safe.




-- 
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.

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



[GitHub] [iotdb] qiaojialin merged pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191


   


-- 
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-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644879183



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
##########
@@ -82,4 +84,20 @@ public Snapshot getSnapshot(long minIndex) {
     snapshot.setLastLogTerm(term);
     return snapshot;
   }
+
+  @Override
+  void applyEntries(List<Log> entries) {

Review comment:
       These codes belong to the transitional code. Originally, it was intended to pass in the information about whether the node is a leader when the metadata group log was applied. Later, this logic was put in the method during optimization, so these codes are useless. I have removed them.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865107395


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D.png' alt='D' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='54.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [54.0% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644121915



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       Why?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644888898



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -73,6 +92,10 @@ public void waitSlot(int slotId) {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > 1000) {

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.

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



[GitHub] [iotdb] LebronAl edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855432111


   > I have added docs already.
   
   Nice~Please pay attention to the timeout CI and fix them
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644913894



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -1023,30 +1157,85 @@ public IReaderByTimestamp getReaderByTimestamp(
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                requiredSlots);

Review comment:
       Yes, I have specified it as null.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645582343



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -767,57 +781,109 @@ public void removeLocalData(List<Integer> slots) {
               SlotPartitionTable.getSlotStrategy()
                   .calculateSlotByPartitionNum(
                       storageGroupName, timePartitionId, ClusterConstant.SLOT_NUM);
+          /**
+           * If this slot is just held by different raft groups in the same node, it should keep the
+           * data of slot.
+           */
+          if (((SlotPartitionTable) metaGroupMember.getPartitionTable())
+              .judgeHoldSlot(thisNode, slot)) {
+            return false;
+          }
           return slotSet.contains(slot);
         };
     for (PartialPath sg : allStorageGroupNames) {
       StorageEngine.getInstance().removePartitions(sg, filter);
     }
     for (Integer slot : slots) {
-      slotManager.setToNull(slot);
+      slotManager.setToNull(slot, false);
     }
+    slotManager.save();
 
     if (logger.isInfoEnabled()) {
       logger.info(
           "{}: data of {} and other {} slots are removed", name, slots.get(0), slots.size() - 1);
     }
   }
 
+  public void preRemoveNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to pre remove node {}", name, removedNode);
+    }
+    synchronized (allNodes) {
+      if (allNodes.contains(removedNode) && allNodes.size() == config.getReplicationNum()) {
+        // update the group if the deleted node was in it
+        PartitionGroup newGroup = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
+        if (newGroup == null) {
+          return;
+        }
+        Node newNodeToGroup = newGroup.get(newGroup.size() - 1);
+        allNodes.add(newNodeToGroup);
+        peerMap.putIfAbsent(newNodeToGroup, new Peer(logManager.getLastLogIndex()));
+      }
+    }
+  }
+
   /**
    * When a node is removed and IT IS NOT THE HEADER of the group, the member should take over some
    * slots from the removed group, and add a new node to the group the removed node was in the
    * group.
    */
   @SuppressWarnings("java:S2445") // the reference of allNodes is unchanged
-  public void removeNode(Node removedNode, NodeRemovalResult removalResult) {
+  public void removeNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to remove node {}", name, removedNode);
+    }
+
     synchronized (allNodes) {
+      preRemoveNode(removedNode);
       if (allNodes.contains(removedNode)) {
         // update the group if the deleted node was in it
-        allNodes = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
-        initPeerMap();
+        allNodes.remove(removedNode);
+        peerMap.remove(removedNode);
         if (removedNode.equals(leader.get())) {
           // if the leader is removed, also start an election immediately
           synchronized (term) {
             setCharacter(NodeCharacter.ELECTOR);
-            setLastHeartbeatReceivedTime(Long.MIN_VALUE);
+            setLeader(null);
+          }
+          synchronized (getHeartBeatWaitObject()) {
+            getHeartBeatWaitObject().notifyAll();
           }
         }
       }
-      List<Integer> slotsToPull =
-          ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
-      if (slotsToPull != null) {
-        // pull the slots that should be taken over
-        PullSnapshotTaskDescriptor taskDescriptor =
-            new PullSnapshotTaskDescriptor(removalResult.getRemovedGroup(), slotsToPull, true);
-        pullFileSnapshot(taskDescriptor, null);
+    }
+  }
+
+  public void pullSlots(NodeRemovalResult removalResult) {
+    List<Integer> slotsToPull =
+        ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
+    if (slotsToPull != null) {
+      // pull the slots that should be taken over
+      PullSnapshotTaskDescriptor taskDescriptor =
+          new PullSnapshotTaskDescriptor(
+              removalResult.getRemovedGroup(getRaftGroupId()), new ArrayList<>(slotsToPull), true);
+      pullFileSnapshot(taskDescriptor, null);
+    }
+  }
+
+  /** For data group, it's necessary to apply remove/add log immediately after append. */
+  @Override
+  protected long appendEntry(long prevLogIndex, long prevLogTerm, long leaderCommit, Log log) {
+    long resp = super.appendEntry(prevLogIndex, prevLogTerm, leaderCommit, log);
+    if (resp == Response.RESPONSE_AGREE
+        && (log instanceof AddNodeLog || log instanceof RemoveNodeLog)) {
+      try {
+        commitLog(log);

Review comment:
       > Is it possible to apply only this log?
   Yes, I have fixed it.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640719205



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
##########
@@ -70,4 +70,14 @@ void sendHeartbeatAsync(Node node) {
     // erase the sent partition table so it will not be sent in the next heartbeat
     request.unsetPartitionTableBytes();
   }
+
+  @Override
+  void startElection() {
+    //    if (localMetaMember.getThisNode().metaPort != 9003 &&
+    // localMetaMember.getThisNode().metaPort != 9005) {
+    //      return;
+    //    }
+    super.startElection();
+    localMetaMember.getAppendLogThreadPool().submit(() -> localMetaMember.processEmptyContentLog());

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639852107



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -79,19 +84,39 @@
   List<PartitionGroup> getLocalGroups();
 
   /**
-   * @param header
+   * @param raftNode
    * @return the partition group starting from the header.
    */
-  PartitionGroup getHeaderGroup(Node header);
+  PartitionGroup getHeaderGroup(RaftNode raftNode);
+
+  PartitionGroup getHeaderGroup(Node node);
 
   ByteBuffer serialize();
 
-  void deserialize(ByteBuffer buffer);
+  /**
+   * Deserialize partition table and check whether the partition table in byte buffer is valid
+   *
+   * @param buffer
+   * @return true if the partition table is valid
+   */
+  boolean deserialize(ByteBuffer buffer);
 
   List<Node> getAllNodes();
 
   List<PartitionGroup> getGlobalGroups();
 
+  List<PartitionGroup> calculateGlobalGroups(List<Node> nodeRing);
+
+  /**
+   * Judge whether the data of slot is held by node
+   *
+   * @param node target node
+   */
+  boolean judgeHoldSlot(Node node, int slot);

Review comment:
       Good suggestion. I have removed it from the interface.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40860830/badge)](https://coveralls.io/builds/40860830)
   
   Coverage increased (+0.2%) to 68.074% when pulling **2248ba1897b1a989e462c972eb49807e054cd65a on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40340900/badge)](https://coveralls.io/builds/40340900)
   
   Coverage decreased (-0.1%) to 67.547% when pulling **ca0228d1f9c2f6c69150fbc153b42e870261ec39 on cluster_scalability** into **11c942a3b5bdcf046c334d79d65b0ba06e84107d on master**.
   


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644149620



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       If the log of this node is not new enough to win an election, then this data group will still have electionTimeout unavailable time?




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40747528/badge)](https://coveralls.io/builds/40747528)
   
   Coverage increased (+0.01%) to 67.898% when pulling **50598277a8c2d977154cea0e764377e9667f1a87 on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-853255706


   > I will review this big feature for three or four stages, and this is the first stage:
   > 
   > BTW, please add the cluster scale-out documents in docs or confluence ASAP~
   
   Please refer to https://cwiki.apache.org/confluence/display/IOTDB/Cluster+scalability+design+doc


-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-844688923


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [127 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='48.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [48.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='2.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640802379



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
##########
@@ -96,19 +96,21 @@ public boolean hasNextDataClient(boolean byTimestamp, long timestamp) {
         Long newReaderId = getReaderId(node, byTimestamp, timestamp);
         if (newReaderId != null) {
           logger.debug("get a readerId {} for {} from {}", newReaderId, request.path, node);
-          if (newReaderId != -1) {
+          if (newReaderId >= 0) {
             // register the node so the remote resources can be released
-            context.registerRemoteNode(node, partitionGroup.getHeader());
+            context.registerRemoteNode(node, partitionGroup.getHeader(), partitionGroup.getId());
             this.readerId = newReaderId;
             this.curSource = node;
             this.curPos = nextNodePos;
             return true;
-          } else {
+          } else if (newReaderId == -1) {
             // the id being -1 means there is no satisfying data on the remote node, create an
             // empty reader to reduce further communication
             this.isNoClient = true;
             this.isNoData = true;
             return false;
+          } else {
+            logger.debug("change other client for better query performance.");

Review comment:
       This is just temporary code,I have rolled back.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
##########
@@ -96,19 +96,21 @@ public boolean hasNextDataClient(boolean byTimestamp, long timestamp) {
         Long newReaderId = getReaderId(node, byTimestamp, timestamp);
         if (newReaderId != null) {
           logger.debug("get a readerId {} for {} from {}", newReaderId, request.path, node);
-          if (newReaderId != -1) {
+          if (newReaderId >= 0) {
             // register the node so the remote resources can be released
-            context.registerRemoteNode(node, partitionGroup.getHeader());
+            context.registerRemoteNode(node, partitionGroup.getHeader(), partitionGroup.getId());
             this.readerId = newReaderId;
             this.curSource = node;
             this.curPos = nextNodePos;
             return true;
-          } else {
+          } else if (newReaderId == -1) {
             // the id being -1 means there is no satisfying data on the remote node, create an
             // empty reader to reduce further communication
             this.isNoClient = true;
             this.isNoData = true;
             return false;
+          } else {
+            logger.debug("change other client for better query performance.");

Review comment:
       This is just temporary codes,I have rolled back.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640777685



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -254,7 +254,7 @@ public void testInstallMultiple()
       List<TsFileResource> loadedFiles = processor.getSequenceFileTreeSet();
       assertEquals(10, loadedFiles.size());
       for (int i = 0; i < 9; i++) {
-        assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
+        assertEquals(-1, loadedFiles.get(i).getMaxPlanIndex());

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639842574



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus();
+        status.setCode(TSStatusCode.NEED_REDIRECTION.getStatusCode());
+      } else {
+        status = StatusUtils.OK;
       }

Review comment:
       I've added the endPoint back.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644941332



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -165,14 +165,14 @@ public static Long querySingleSeries(
   }
 
   public static List<String> getNodeList(

Review comment:
       Because these two methods use `SingleSeriesQueryRequest`, I have added `raftId` into this request. So it's unnecessary to add another parameter in methods.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645499574



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode());

Review comment:
       As you wish.




-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648779379



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.getCoordinator().sendLogToAllDataGroups(log);

Review comment:
       If a slot is owned by B datagroup, it will be moved to A datagroup in the new partition table. If A datagroup performs this step first, it will start a new datagroupmember to receive new writes. However, B datagroup still has a follower that has not been implemented to this step. If a write request is sent to it, it will route to the leader of the raft group, and the leader will execute it directly. When A datagroup pulls a slot later, it may lose some data?




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852290094






-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865197014


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D.png' alt='D' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='53.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [53.6% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644860167



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       When the leader of a data group is removed from the cluster, the data group should elect a new leader.  Each node in the data group will be an elector and try to be the leader. If the log of this node is not new enough to win an election, it just will not win the election and the node with the newest log will win. Why data group will still have electionTimeout unavailable time?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639844152



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
##########
@@ -173,23 +181,35 @@ private boolean pullSnapshot(int nodeIndex) throws InterruptedException, TExcept
 
   @Override
   public Void call() {
-    persistTask();
     request = new PullSnapshotRequest();
     request.setHeader(descriptor.getPreviousHolders().getHeader());
+    request.setRaftId(descriptor.getPreviousHolders().getId());
     request.setRequiredSlots(descriptor.getSlots());
     request.setRequireReadOnly(descriptor.isRequireReadOnly());
 
+    logger.info("{}: data migration starts.", newMember.getName());
     boolean finished = false;
-    int nodeIndex = -1;
+    int nodeIndex = ((PartitionGroup) newMember.getAllNodes()).indexOf(newMember.getThisNode()) - 1;
     while (!finished) {
       try {
         // sequentially pick up a node that may have this slot
         nodeIndex = (nodeIndex + 1) % descriptor.getPreviousHolders().size();
+        long startTime = System.currentTimeMillis();
         finished = pullSnapshot(nodeIndex);
         if (!finished) {
+          if (logger.isDebugEnabled()) {
+            logger.debug(
+                "Cannot pull slot {} from {}, retry",
+                descriptor.getSlots(),
+                descriptor.getPreviousHolders().get(nodeIndex));
+          }
           Thread.sleep(
               ClusterDescriptor.getInstance().getConfig().getPullSnapshotRetryIntervalMs());
         }
+        logger.debug(
+            "{}: Data migration ends, cost {}ms",
+            newMember,
+            (System.currentTimeMillis() - startTime));

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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-867041752


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [130 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='52.8%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [52.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.2% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644871898



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -82,11 +120,17 @@ public boolean equals(Object o) {
       return false;
     }
     AddNodeLog that = (AddNodeLog) o;
-    return Objects.equals(newNode, that.newNode);
+    return Objects.equals(newNode, that.newNode)

Review comment:
       Because sub-class has its own variables, it's necessary to rewrite these two functions for completeness. All sub-class of class Log have rewritten these two functions.




-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r643918392



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)
+        throws SnapshotInstallationException {
       try {
         logger.info("Starting to install a snapshot {} into slot[{}]", snapshot, slot);
         installFileSnapshotSchema(snapshot);
         logger.info("Schemas in snapshot are registered");
-
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as the schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as the schemas are set, writes can proceed
+            slotManager.setToPullingWritable(slot);
+            logger.debug("{}: slot {} is now pulling writable", name, slot);
+          }
         }
-
-        installFileSnapshotFiles(snapshot, slot);
+        installFileSnapshotFiles(snapshot, slot, isDataMigration);
       } catch (PullFileException e) {
         throw new SnapshotInstallationException(e);
       }
     }
 
     @Override
-    public void install(Map<Integer, FileSnapshot> snapshotMap)
+    public void install(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
       logger.info("Starting to install snapshots {}", snapshotMap);
-      installSnapshot(snapshotMap);
+      installSnapshot(snapshotMap, isDataMigration);
     }
 
-    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap)
+    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
-      // ensure StorageGroups are synchronized
-      try {
-        dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(true);
-      } catch (CheckConsistencyException e) {
-        throw new SnapshotInstallationException(e);
-      }
-
-      for (FileSnapshot value : snapshotMap.values()) {
-        installFileSnapshotSchema(value);
-      }
-
+      // In data migration, meta group member other than new node does not need to synchronize the
+      // leader, because data migration must be carried out after meta group applied add/remove node
+      // log.
+      dataGroupMember
+          .getMetaGroupMember()
+          .syncLocalApply(
+              dataGroupMember.getMetaGroupMember().getPartitionTable().getLastMetaLogIndex() - 1,
+              false);
       for (Entry<Integer, FileSnapshot> integerSnapshotEntry : snapshotMap.entrySet()) {
         Integer slot = integerSnapshotEntry.getKey();
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        FileSnapshot snapshot = integerSnapshotEntry.getValue();
+        installFileSnapshotSchema(snapshot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as schemas are set, writes can proceed

Review comment:
       comment is not right

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
##########
@@ -677,6 +677,9 @@ public boolean matchTerm(long term, long index) {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
+      if (entry.isApplied()) {

Review comment:
       Is this judgment useful? When will apply a log that has been applied(except for restart)?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644911409



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -86,16 +109,15 @@ public void waitSlot(int slotId) {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= 5000) {

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.

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



[GitHub] [iotdb] LebronAl commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r643798830



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       It seems that we should sync leader first to make this node catch up and then try to start an election?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -32,25 +32,57 @@
 /** AddNodeLog records the operation of adding a node into this cluster. */
 public class AddNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node newNode;
 
-  public Node getNewNode() {
-    return newNode;
+  private long metaLogIndex;

Review comment:
       do we really need this field? It seems that it is always equal to currLogIndex?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)

Review comment:
       It looks like the isDataIntegration parameter is all false in the callers of this function

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -86,16 +109,15 @@ public void waitSlot(int slotId) {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= 5000) {

Review comment:
       same as above

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -1023,30 +1157,85 @@ public IReaderByTimestamp getReaderByTimestamp(
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                requiredSlots);

Review comment:
       always null?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
##########
@@ -82,4 +84,20 @@ public Snapshot getSnapshot(long minIndex) {
     snapshot.setLastLogTerm(term);
     return snapshot;
   }
+
+  @Override
+  void applyEntries(List<Log> entries) {

Review comment:
       What is the difference between this function and its parent class? Is there a corner case? You'd better add some comments

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -82,11 +120,17 @@ public boolean equals(Object o) {
       return false;
     }
     AddNodeLog that = (AddNodeLog) o;
-    return Objects.equals(newNode, that.newNode);
+    return Objects.equals(newNode, that.newNode)

Review comment:
       The base class `Log` use `currLogIndex` and `currLogTerm` for `equals` and `hashcode`. What's the purpose of rewriting these two functions here?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
##########
@@ -677,6 +677,9 @@ public boolean matchTerm(long term, long index) {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
+      if (entry.isApplied()) {

Review comment:
       Please add some comments

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);

Review comment:
       add comments

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -73,6 +92,10 @@ public void waitSlot(int slotId) {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > 1000) {

Review comment:
       do not use magic number, make it a final field

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);
 
   /**
    * Remove a node and update the partition table.
    *
    * @param node
    */
-  NodeRemovalResult removeNode(Node node);
+  void removeNode(Node node);
+
+  NodeRemovalResult getNodeRemovalResult();

Review comment:
       add comments

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -77,11 +115,17 @@ public boolean equals(Object o) {
       return false;
     }
     RemoveNodeLog that = (RemoveNodeLog) o;
-    return Objects.equals(removedNode, that.removedNode);
+    return Objects.equals(removedNode, that.removedNode)

Review comment:
       same as above

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -31,17 +31,49 @@
 
 public class RemoveNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node removedNode;
 
+  private long metaLogIndex;

Review comment:
       same as above

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -507,11 +541,16 @@ private SeriesReader getSeriesReader(
       Filter valueFilter,
       QueryContext context,
       Node header,
-      boolean ascending)
+      int raftId,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    if (requiredSlots == null) {

Review comment:
       Please add some comments for this if statement.
   BTW, why don't judge this in other functions such as `getMultSeriesReader`?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -337,7 +362,8 @@ private AbstractMultPointReader getMultSeriesReader(
           deviceMeasurements,
           partitionGroup,
           context,
-          ascending);
+          ascending,
+          requiredSlots);

Review comment:
       always NULL?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node

Review comment:
       update comments




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-867041752


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [130 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='52.8%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [52.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.2% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r643614637



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode());

Review comment:
       maybe these two lines can be wrapped into a function in `StatusUtils`

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -890,4 +988,57 @@ public boolean isUnchanged() {
   public void setUnchanged(boolean unchanged) {
     this.unchanged = unchanged;
   }
+
+  public void setAndSaveLastAppliedPartitionTableVersion(long version) {
+    lastAppliedPartitionTableVersion.setVersion(version);
+    lastAppliedPartitionTableVersion.save();
+  }
+
+  private class LastAppliedPatitionTableVersion {
+
+    private static final String VERSION_FILE_NAME = "LAST_PARTITION_TABLE_VERSION";
+
+    private long version = -1;
+
+    private String filePath;
+
+    public LastAppliedPatitionTableVersion(String memberDir) {
+      this.filePath = memberDir + File.separator + VERSION_FILE_NAME;
+      load();
+    }
+
+    private void load() {
+      File versionFile = new File(filePath);
+      if (!versionFile.exists()) {
+        return;
+      }
+      try (FileInputStream fileInputStream = new FileInputStream(filePath);
+          DataInputStream dataInputStream = new DataInputStream(fileInputStream)) {
+        version = dataInputStream.readLong();
+      } catch (Exception e) {
+        logger.warn("Cannot deserialize last partition table version from {}", filePath, e);
+      }
+    }
+
+    public synchronized void save() {

Review comment:
       this should be atomic? maybe we need a atomic-rename mechanism

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -165,14 +165,14 @@ public static Long querySingleSeries(
   }
 
   public static List<String> getNodeList(

Review comment:
       Why `querySingleSeriesByTimestamp` and  `querySingleSeries` function don't have `raftId`? It seems that these query function's behavior should be consistent?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -106,13 +106,13 @@ public static Long removeNode(AsyncMetaClient asyncMetaClient, Node nodeToRemove
   }
 
   public static Boolean matchTerm(
-      AsyncClient client, Node target, long prevLogIndex, long prevLogTerm, Node header)

Review comment:
       As there already has a RaftNode struct in `cluster.thrift`, maybe we can change these rpc parameters from `Node header,int raftId` to `RaftNode raftNode`, this will be much clearer

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -435,32 +440,42 @@ public static TSStatus executeNonQuery(
   }
 
   public static ByteBuffer readFile(
-      AsyncDataClient client, String remotePath, long offset, int fetchSize)
+      AsyncDataClient client, String remotePath, long offset, int fetchSize, int raftId)

Review comment:
       Maybe `raftId` is not needed in `readFile` rpc and some other rpcs?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640759465



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/utils/nodetool/ClusterMonitor.java
##########
@@ -66,9 +77,67 @@ public void start() throws StartupException {
   }
 
   @Override
+  public List<Pair<Node, NodeCharacter>> getMetaGroup() {
+    MetaGroupMember metaMember = getMetaGroupMember();
+    if (metaMember == null || metaMember.getPartitionTable() == null) {
+      return null;
+    }
+    List<Pair<Node, NodeCharacter>> res = new ArrayList<>();
+    for (Node node : metaMember.getPartitionTable().getAllNodes()) {
+      if (node.equals(metaMember.getLeader())) {
+        res.add(new Pair<>(node, NodeCharacter.LEADER));
+      } else {
+        res.add(new Pair<>(node, NodeCharacter.FOLLOWER));
+      }
+    }
+    return res;

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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-843750763


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E.png' alt='E' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E.png' alt='E' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [160 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='46.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [46.2% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40313409/badge)](https://coveralls.io/builds/40313409)
   
   Coverage increased (+0.1%) to 68.035% when pulling **292cf2318eb4ddd49951bf9f05a2fe8b689689ee on cluster_scalability** into **16167373d924b4f457ce380144a2564be76b65b2 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648483387



##########
File path: cluster/src/assembly/resources/sbin/remove-node.bat
##########
@@ -0,0 +1,117 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+echo ````````````````````````
+echo Starting to remove a node (Cluster Mode)
+echo ````````````````````````
+
+PATH %PATH%;%JAVA_HOME%\bin\
+set "FULL_VERSION="
+set "MAJOR_VERSION="
+set "MINOR_VERSION="
+
+
+for /f tokens^=2-5^ delims^=.-_+^" %%j in ('java -fullversion 2^>^&1') do (
+	set "FULL_VERSION=%%j-%%k-%%l-%%m"
+	IF "%%j" == "1" (
+	    set "MAJOR_VERSION=%%k"
+	    set "MINOR_VERSION=%%l"
+	) else (
+	    set "MAJOR_VERSION=%%j"
+	    set "MINOR_VERSION=%%k"
+	)
+)
+
+set JAVA_VERSION=%MAJOR_VERSION%
+
+IF NOT %JAVA_VERSION% == 8 (
+	IF NOT %JAVA_VERSION% == 11 (
+		echo IoTDB only supports jdk8 or jdk11, please check your java version.
+		goto finally
+	)
+)
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
+popd
+
+set IOTDB_CONF=%IOTDB_HOME%\conf
+set IOTDB_LOGS=%IOTDB_HOME%\logs
+
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-r
+set is_conf_path=false
+for %%i in (%*) do (
+	IF "%%i" == "-c" (
+		set is_conf_path=true
+	) ELSE IF "!is_conf_path!" == "true" (

Review comment:
       fixed

##########
File path: cluster/src/assembly/resources/sbin/remove-node.sh
##########
@@ -0,0 +1,88 @@
+#!/bin/bash
+#
+# 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.
+#
+
+
+echo ---------------------
+echo "Starting to remove a node(Cluster Mode)"
+echo ---------------------
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="`dirname "$0"`/.."
+fi
+
+IOTDB_CONF=${IOTDB_HOME}/conf
+
+is_conf_path=false
+for arg do
+    shift
+    if [ "$arg" == "-c" ]; then
+        is_conf_path=true
+        continue

Review comment:
       fixed

##########
File path: cluster/src/assembly/resources/sbin/remove-node.bat
##########
@@ -0,0 +1,117 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+echo ````````````````````````
+echo Starting to remove a node (Cluster Mode)
+echo ````````````````````````
+
+PATH %PATH%;%JAVA_HOME%\bin\
+set "FULL_VERSION="
+set "MAJOR_VERSION="
+set "MINOR_VERSION="
+
+
+for /f tokens^=2-5^ delims^=.-_+^" %%j in ('java -fullversion 2^>^&1') do (
+	set "FULL_VERSION=%%j-%%k-%%l-%%m"
+	IF "%%j" == "1" (
+	    set "MAJOR_VERSION=%%k"
+	    set "MINOR_VERSION=%%l"
+	) else (
+	    set "MAJOR_VERSION=%%j"
+	    set "MINOR_VERSION=%%k"
+	)
+)
+
+set JAVA_VERSION=%MAJOR_VERSION%
+
+IF NOT %JAVA_VERSION% == 8 (
+	IF NOT %JAVA_VERSION% == 11 (
+		echo IoTDB only supports jdk8 or jdk11, please check your java version.
+		goto finally
+	)
+)
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
+popd
+
+set IOTDB_CONF=%IOTDB_HOME%\conf
+set IOTDB_LOGS=%IOTDB_HOME%\logs
+
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-r
+set is_conf_path=false
+for %%i in (%*) do (
+	IF "%%i" == "-c" (
+		set is_conf_path=true
+	) ELSE IF "!is_conf_path!" == "true" (

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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639877990



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java
##########
@@ -1175,16 +1210,16 @@ public void testRemoteAddNode() {
       assertTrue(response.getCheckStatusResponse().isClusterNameEquals());
 
       // cannot add a node due to network failure
-      dummyResponse.set(Response.RESPONSE_NO_CONNECTION);
-      testMetaMember.setCharacter(LEADER);
-      result.set(null);
-      testMetaMember.setPartitionTable(partitionTable);
-      new Thread(
-              () -> {
-                await().atLeast(200, TimeUnit.MILLISECONDS);
-                dummyResponse.set(Response.RESPONSE_AGREE);
-              })
-          .start();
+      //      dummyResponse.set(Response.RESPONSE_NO_CONNECTION);
+      //      testMetaMember.setCharacter(LEADER);
+      //      result.set(null);
+      //      testMetaMember.setPartitionTable(partitionTable);
+      //      new Thread(
+      //              () -> {
+      //                await().atLeast(200, TimeUnit.MILLISECONDS);
+      //                dummyResponse.set(Response.RESPONSE_AGREE);
+      //              })
+      //          .start();

Review comment:
       Resume.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639833332



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,46 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log)
+      throws ChangeMembershipException, InterruptedException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }

Review comment:
       Yes, the add/remove log for the data group will trigger pre add/remove node operation which is idempotent.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40911484/badge)](https://coveralls.io/builds/40911484)
   
   Coverage decreased (-0.03%) to 67.865% when pulling **6a38a46bba9c64857017f288850c673a485c3567 on cluster_scalability** into **affe6d95ec3ddaf0689e58d0283945ac29d54830 on master**.
   


-- 
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-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852290094


   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=coverage&view=list) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo.png' alt='No Duplication information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

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



[GitHub] [iotdb] jt2594838 commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-841101679


   Please attach your design document or a link to it, I would recommend anyone read it before proceeding to the code review.


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644889526



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -86,16 +109,15 @@ public void waitSlot(int slotId) {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= 5000) {

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.

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



[GitHub] [iotdb] LebronAl commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645592393



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(

Review comment:
       Yeah, this has nothing to do with your PR, it's so nice that you can fix this. :)




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-843750763


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E.png' alt='E' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E.png' alt='E' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [160 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='46.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [46.2% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644885703



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);
 
   /**
    * Remove a node and update the partition table.
    *
    * @param node
    */
-  NodeRemovalResult removeNode(Node node);
+  void removeNode(Node node);
+
+  NodeRemovalResult getNodeRemovalResult();

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.

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



[GitHub] [iotdb] fanhualta commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-853256335


   > > Please attach your design document or a link to it, I would recommend anyone read it before proceeding to the code review.
   > 
   > So, where is the cluster scalability design document? I only see a document about multi-raft in https://cwiki.apache.org/confluence/display/IOTDB/Multi-Raft-For-One-DataGroup+Design+Doc
   
   Please refer to 
   Please refer to https://cwiki.apache.org/confluence/display/IOTDB/Cluster+scalability+design+doc


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644883564



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
##########
@@ -677,6 +677,9 @@ public boolean matchTerm(long term, long index) {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
+      if (entry.isApplied()) {

Review comment:
       I have added these comments: 
   For add/remove logs in data groups, this log will be applied immediately when it is appended to the raft log. In this case, it will apply a log that has been applied.




-- 
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.

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



[GitHub] [iotdb] fanhualta closed pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta closed pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191


   


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-842166665


   Hi, I think it's time for us to start adding cluster design documents to the docs now~


-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r646238962



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);

Review comment:
       If a slot is owned by B datagroup, it will be moved to A datagroup in the new partition table. If A datagroup performs this step first, it will start a new datagroupmember to receive new writes. However, B datagroup still has a follower that has not been implemented to this step. If a write request is sent to it, it will route to the leader of the raft group, and the leader will execute it directly. When A datagroup pulls a slot later, it may lose some data?




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865197014


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D.png' alt='D' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='53.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [53.6% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r643918392



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)
+        throws SnapshotInstallationException {
       try {
         logger.info("Starting to install a snapshot {} into slot[{}]", snapshot, slot);
         installFileSnapshotSchema(snapshot);
         logger.info("Schemas in snapshot are registered");
-
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as the schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as the schemas are set, writes can proceed
+            slotManager.setToPullingWritable(slot);
+            logger.debug("{}: slot {} is now pulling writable", name, slot);
+          }
         }
-
-        installFileSnapshotFiles(snapshot, slot);
+        installFileSnapshotFiles(snapshot, slot, isDataMigration);
       } catch (PullFileException e) {
         throw new SnapshotInstallationException(e);
       }
     }
 
     @Override
-    public void install(Map<Integer, FileSnapshot> snapshotMap)
+    public void install(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
       logger.info("Starting to install snapshots {}", snapshotMap);
-      installSnapshot(snapshotMap);
+      installSnapshot(snapshotMap, isDataMigration);
     }
 
-    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap)
+    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
-      // ensure StorageGroups are synchronized
-      try {
-        dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(true);
-      } catch (CheckConsistencyException e) {
-        throw new SnapshotInstallationException(e);
-      }
-
-      for (FileSnapshot value : snapshotMap.values()) {
-        installFileSnapshotSchema(value);
-      }
-
+      // In data migration, meta group member other than new node does not need to synchronize the
+      // leader, because data migration must be carried out after meta group applied add/remove node
+      // log.
+      dataGroupMember
+          .getMetaGroupMember()
+          .syncLocalApply(
+              dataGroupMember.getMetaGroupMember().getPartitionTable().getLastMetaLogIndex() - 1,
+              false);
       for (Entry<Integer, FileSnapshot> integerSnapshotEntry : snapshotMap.entrySet()) {
         Integer slot = integerSnapshotEntry.getKey();
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        FileSnapshot snapshot = integerSnapshotEntry.getValue();
+        installFileSnapshotSchema(snapshot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as schemas are set, writes can proceed

Review comment:
       comment is not right




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855427640


   > Please add some user guide in `https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster` &&`https://github.com/apache/iotdb/tree/master/docs/UserGuide/Cluster` and add some examples for add/remove node operation
   
   I have added docs already.


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644944171



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -890,4 +988,57 @@ public boolean isUnchanged() {
   public void setUnchanged(boolean unchanged) {
     this.unchanged = unchanged;
   }
+
+  public void setAndSaveLastAppliedPartitionTableVersion(long version) {
+    lastAppliedPartitionTableVersion.setVersion(version);
+    lastAppliedPartitionTableVersion.save();
+  }
+
+  private class LastAppliedPatitionTableVersion {
+
+    private static final String VERSION_FILE_NAME = "LAST_PARTITION_TABLE_VERSION";
+
+    private long version = -1;
+
+    private String filePath;
+
+    public LastAppliedPatitionTableVersion(String memberDir) {
+      this.filePath = memberDir + File.separator + VERSION_FILE_NAME;
+      load();
+    }
+
+    private void load() {
+      File versionFile = new File(filePath);
+      if (!versionFile.exists()) {
+        return;
+      }
+      try (FileInputStream fileInputStream = new FileInputStream(filePath);
+          DataInputStream dataInputStream = new DataInputStream(fileInputStream)) {
+        version = dataInputStream.readLong();
+      } catch (Exception e) {
+        logger.warn("Cannot deserialize last partition table version from {}", filePath, e);
+      }
+    }
+
+    public synchronized void save() {

Review comment:
       Maybe unnecessary, I implemented it in imitation of other places, like `SlotManger`.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639874870



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -2605,39 +2605,61 @@ private boolean loadTsFileByType(
       targetFile.getParentFile().mkdirs();
     }
     try {
-      FileUtils.moveFile(syncedTsFile, targetFile);
+      FileUtils.moveFile(tsFileToLoad, targetFile);
     } catch (IOException e) {
       logger.error(
           "File renaming failed when loading tsfile. Origin: {}, Target: {}",
-          syncedTsFile.getAbsolutePath(),
+          tsFileToLoad.getAbsolutePath(),
           targetFile.getAbsolutePath(),
           e);
       throw new LoadFileException(
           String.format(
               "File renaming failed when loading tsfile. Origin: %s, Target: %s, because %s",
-              syncedTsFile.getAbsolutePath(), targetFile.getAbsolutePath(), e.getMessage()));
+              tsFileToLoad.getAbsolutePath(), targetFile.getAbsolutePath(), e.getMessage()));
     }
 
-    File syncedResourceFile =
-        fsFactory.getFile(syncedTsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    File resourceFileToLoad =
+        fsFactory.getFile(tsFileToLoad.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
     File targetResourceFile =
         fsFactory.getFile(targetFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
     try {
-      FileUtils.moveFile(syncedResourceFile, targetResourceFile);
+      FileUtils.moveFile(resourceFileToLoad, targetResourceFile);
     } catch (IOException e) {
       logger.error(
           "File renaming failed when loading .resource file. Origin: {}, Target: {}",
-          syncedResourceFile.getAbsolutePath(),
+          resourceFileToLoad.getAbsolutePath(),
           targetResourceFile.getAbsolutePath(),
           e);
       throw new LoadFileException(
           String.format(
               "File renaming failed when loading .resource file. Origin: %s, Target: %s, because %s",
-              syncedResourceFile.getAbsolutePath(),
+              resourceFileToLoad.getAbsolutePath(),
               targetResourceFile.getAbsolutePath(),
               e.getMessage()));
     }
 
+    File modFileToLoad =
+        fsFactory.getFile(tsFileToLoad.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    if (modFileToLoad.exists()) {
+      File targetModFile =
+          fsFactory.getFile(targetFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);

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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-849847242


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [127 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='49.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [49.2% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='2.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [2.3% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639800481



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -499,18 +515,26 @@ public static ByteBuffer last(
             context.getQueryId(),
             deviceMeasurements,
             header,
+            raftId,
             client.getNode());
 
     client.last(request, handler);
     return handler.getResult(RaftServer.getReadOperationTimeoutMS());
   }
 
-  public static boolean onSnapshotApplied(AsyncDataClient client, Node header, List<Integer> slots)
+  public static boolean onSnapshotApplied(
+      AsyncDataClient client, Node header, int raftId, List<Integer> slots)
       throws TException, InterruptedException {
-    AtomicReference<Boolean> result = new AtomicReference<>(false);
+    AtomicReference<Boolean> result = new AtomicReference<>();
     GenericHandler<Boolean> handler = new GenericHandler<>(client.getNode(), result);
 
-    client.onSnapshotApplied(header, slots, handler);
-    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+    client.onSnapshotApplied(header, raftId, slots, handler);
+
+    synchronized (result) {
+      if (result.get() == null) {
+        result.wait();
+      }
+    }

Review comment:
       Rollback to add time-out mechanism.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-849847242


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [127 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40.png' alt='49.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [49.2% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='2.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [2.3% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818






-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855154168


   My review has almost been completed. Next, please do two things:
   - Fix CI
   - Add some user guide in https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster &&https://github.com/apache/iotdb/tree/master/docs/UserGuide/Cluster and add some examples for add/remove node operation


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855432111


   > I have added docs already.
   Nice~Please pay attention to the timeout CI and fix them
   


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-854297217


   Please add some user guide in `https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster` and add some examples for add/remove node operation


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639882219



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/LogParserTest.java
##########
@@ -98,4 +104,14 @@ public void testEmptyContentLog() throws UnknownLogTypeException {
     Log serialized = logParser.parse(byteBuffer);
     assertEquals(log, serialized);
   }
+
+  @Test
+  public void testLogPlan() throws IOException, IllegalPathException, UnknownLogTypeException {
+    AddNodeLog log = new AddNodeLog(TestUtils.seralizePartitionTable, TestUtils.getNode(0));
+    log.setMetaLogIndex(1);
+    LogPlan logPlan = new LogPlan(log.serialize());
+    ByteBuffer buffer = ByteBuffer.wrap(PlanSerializer.getInstance().serialize(logPlan));
+    PhysicalPlan plan = PhysicalPlan.Factory.create(buffer);
+    LogParser.getINSTANCE().parse(((LogPlan) plan).getLog());
+  }

Review comment:
       Add fail() if exception occurs.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640814498



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
##########
@@ -919,10 +948,18 @@ protected boolean waitUntilCatchUp(CheckConsistency checkConsistency)
    * @param leaderCommitId leader commit id
    * @return true if leaderCommitId <= localAppliedId
    */
-  private boolean syncLocalApply(long leaderCommitId) {
+  public boolean syncLocalApply(long leaderCommitId) {
     long startTime = System.currentTimeMillis();
     long waitedTime = 0;
     long localAppliedId = 0;
+
+    // If the leader and follower logs differ too much, local query is not allowed
+    if (leaderCommitId - logManager.getMaxHaveAppliedCommitIndex() > 1000) {
+      logger.info(
+          "{}: The raft log of this member is too backward to provide service directly.", name);
+      return false;
+    }

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.

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



[GitHub] [iotdb] fanhualta removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-855427551


   > My review has almost been completed. Next, please do two things:
   > 
   > * Fix CI
   > * Add some user guide in https://github.com/apache/iotdb/tree/master/docs/zh/UserGuide/Cluster &&https://github.com/apache/iotdb/tree/master/docs/UserGuide/Cluster and add some examples for add/remove node operation
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645568590



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(

Review comment:
       In the master branch, these functions are moved to the `Coordinator`.  When merge master, these useless functions are left behind. I have removed these useless functions.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {

Review comment:
       Same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(

Review comment:
       Same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(

Review comment:
       Same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {

Review comment:
       Same as above.




-- 
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.

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



[GitHub] [iotdb] mychaow commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r646238962



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
##########
@@ -48,22 +47,44 @@ public void apply(Log log) {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);

Review comment:
       If a slot is owned by B datagroup, it will be moved to a datagroup in the new partition table. If a datagroup performs this step first, it will start a new datagroupmember to receive new writes. However, B datagroup still has a follower that has not been implemented to this step. If a write request is sent to it, it will route to the leader of the raft group, and the leader will execute it directly. When a datagroup pulls a slot later, it may lose some data?




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644925946



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -507,11 +541,16 @@ private SeriesReader getSeriesReader(
       Filter valueFilter,
       QueryContext context,
       Node header,
-      boolean ascending)
+      int raftId,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    if (requiredSlots == null) {

Review comment:
       I have added comments.
   `why don't judge this in other functions such as getMultSeriesReader?`
   Because all other readers will call this function eventually.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644860167



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -314,97 +321,61 @@ public synchronized boolean addNode(Node node, NodeAdditionResult result) {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(new RaftNode(getHeader(), getRaftGroupId()), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);

Review comment:
       When the leader of a data group is removed from the cluster, the data group should elect a new leader.  Each node in the data group will be an elector and try to be the leader. If the log of this node is not new enough to win an election, it just will not win the election and the node with the newest log will win. Why data group will still have electionTimeout unavailable time?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -32,25 +32,57 @@
 /** AddNodeLog records the operation of adding a node into this cluster. */
 public class AddNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node newNode;
 
-  public Node getNewNode() {
-    return newNode;
+  private long metaLogIndex;

Review comment:
       In the raft log of the data group, currLogIndex means the index of the raft log in the data group, but metaLogIndex means the index of the raft log in the meta group.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -31,17 +31,49 @@
 
 public class RemoveNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node removedNode;
 
+  private long metaLogIndex;

Review comment:
       The answer is the same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -82,11 +120,17 @@ public boolean equals(Object o) {
       return false;
     }
     AddNodeLog that = (AddNodeLog) o;
-    return Objects.equals(newNode, that.newNode);
+    return Objects.equals(newNode, that.newNode)

Review comment:
       Because sub-class has its own variables, it's necessary to rewrite these two functions for completeness. All sub-class of class Log have rewritten these two functions.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -77,11 +115,17 @@ public boolean equals(Object o) {
       return false;
     }
     RemoveNodeLog that = (RemoveNodeLog) o;
-    return Objects.equals(removedNode, that.removedNode);
+    return Objects.equals(removedNode, that.removedNode)

Review comment:
       The answer is the same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
##########
@@ -82,4 +84,20 @@ public Snapshot getSnapshot(long minIndex) {
     snapshot.setLastLogTerm(term);
     return snapshot;
   }
+
+  @Override
+  void applyEntries(List<Log> entries) {

Review comment:
       These codes belong to the transitional code. Originally, it was intended to pass in the information about whether the node is a leader when the metadata group log was applied. Later, this logic was put in the method during optimization, so these codes are useless. I have removed them.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
##########
@@ -195,64 +194,70 @@ public String toString() {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)

Review comment:
       Yes, but the parameter is necessary. With this parameter, this method is more complete.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
##########
@@ -677,6 +677,9 @@ public boolean matchTerm(long term, long index) {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
+      if (entry.isApplied()) {

Review comment:
       I have added these comments: 
   For add/remove logs in data groups, this log will be applied immediately when it is appended to the raft log. In this case, it will apply a log that has been applied.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);
 
   /**
    * Remove a node and update the partition table.
    *
    * @param node
    */
-  NodeRemovalResult removeNode(Node node);
+  void removeNode(Node node);
+
+  NodeRemovalResult getNodeRemovalResult();

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -73,6 +92,10 @@ public void waitSlot(int slotId) {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > 1000) {

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -86,16 +109,15 @@ public void waitSlot(int slotId) {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= 5000) {

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -73,6 +92,10 @@ public void waitSlot(int slotId) {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > 1000) {

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
##########
@@ -86,16 +109,15 @@ public void waitSlot(int slotId) {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= 5000) {

Review comment:
       fixed

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -1023,30 +1157,85 @@ public IReaderByTimestamp getReaderByTimestamp(
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                requiredSlots);

Review comment:
       Yes. I have specified it as null.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -1023,30 +1157,85 @@ public IReaderByTimestamp getReaderByTimestamp(
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                requiredSlots);

Review comment:
       Yes, I have specified it as null.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -507,11 +541,16 @@ private SeriesReader getSeriesReader(
       Filter valueFilter,
       QueryContext context,
       Node header,
-      boolean ascending)
+      int raftId,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    if (requiredSlots == null) {

Review comment:
       I have added comments.
   `why don't judge this in other functions such as getMultSeriesReader?`
   Because all other readers will call this function eventually.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -337,7 +362,8 @@ private AbstractMultPointReader getMultSeriesReader(
           deviceMeasurements,
           partitionGroup,
           context,
-          ascending);
+          ascending,
+          requiredSlots);

Review comment:
       I have checked the logic and remove useless parameters.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -507,11 +541,16 @@ private SeriesReader getSeriesReader(
       Filter valueFilter,
       QueryContext context,
       Node header,
-      boolean ascending)
+      int raftId,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    if (requiredSlots == null) {

Review comment:
       I have added comments.
   
   > why don't judge this in other functions such as getMultSeriesReader?
   
   
   Because all other readers will call this function eventually.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -442,9 +445,11 @@ private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGr
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode());

Review comment:
       StatusUtils doesn't have a suitable static method. Although I can use `StatusUtils.getStatus(new TSStatus(TSStatusCode.NEED_REDIRECTION.getStatusCode()), endPoint)`, it will call one more deepClone(). I prefer not to do this only for save one line.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -165,14 +165,14 @@ public static Long querySingleSeries(
   }
 
   public static List<String> getNodeList(

Review comment:
       Because these two methods use `SingleSeriesQueryRequest`, I have added `raftId` into this request. So it's unnecessary to add another parameter in methods.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -890,4 +988,57 @@ public boolean isUnchanged() {
   public void setUnchanged(boolean unchanged) {
     this.unchanged = unchanged;
   }
+
+  public void setAndSaveLastAppliedPartitionTableVersion(long version) {
+    lastAppliedPartitionTableVersion.setVersion(version);
+    lastAppliedPartitionTableVersion.save();
+  }
+
+  private class LastAppliedPatitionTableVersion {
+
+    private static final String VERSION_FILE_NAME = "LAST_PARTITION_TABLE_VERSION";
+
+    private long version = -1;
+
+    private String filePath;
+
+    public LastAppliedPatitionTableVersion(String memberDir) {
+      this.filePath = memberDir + File.separator + VERSION_FILE_NAME;
+      load();
+    }
+
+    private void load() {
+      File versionFile = new File(filePath);
+      if (!versionFile.exists()) {
+        return;
+      }
+      try (FileInputStream fileInputStream = new FileInputStream(filePath);
+          DataInputStream dataInputStream = new DataInputStream(fileInputStream)) {
+        version = dataInputStream.readLong();
+      } catch (Exception e) {
+        logger.warn("Cannot deserialize last partition table version from {}", filePath, e);
+      }
+    }
+
+    public synchronized void save() {

Review comment:
       Maybe unnecessary, I implemented it in imitation of other places, like `SlotManger`.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -435,32 +440,42 @@ public static TSStatus executeNonQuery(
   }
 
   public static ByteBuffer readFile(
-      AsyncDataClient client, String remotePath, long offset, int fetchSize)
+      AsyncDataClient client, String remotePath, long offset, int fetchSize, int raftId)

Review comment:
       I have removed raft from the following methods: 
   
   - readFile
   - removeHardLink
   
   Because these two methods are unassociated with raft group.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
##########
@@ -106,13 +106,13 @@ public static Long removeNode(AsyncMetaClient asyncMetaClient, Node nodeToRemove
   }
 
   public static Boolean matchTerm(
-      AsyncClient client, Node target, long prevLogIndex, long prevLogTerm, Node header)

Review comment:
       I have replaced all `Node header,int raftId` with `RaftNode raftNode`.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40911156/badge)](https://coveralls.io/builds/40911156)
   
   Coverage decreased (-0.03%) to 67.859% when pulling **300169aa91cc5f646dd02ed8a7ffb56c2edc3f9a on cluster_scalability** into **affe6d95ec3ddaf0689e58d0283945ac29d54830 on master**.
   


-- 
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-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644866475



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
##########
@@ -32,25 +32,57 @@
 /** AddNodeLog records the operation of adding a node into this cluster. */
 public class AddNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node newNode;
 
-  public Node getNewNode() {
-    return newNode;
+  private long metaLogIndex;

Review comment:
       In the raft log of the data group, currLogIndex means the index of the raft log in the data group, but metaLogIndex means the index of the raft log in the meta group.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640790678



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -301,6 +301,6 @@ public void testInstallPartial()
     for (int i = 0; i < 9; i++) {
       assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
     }
-    assertEquals(0, processor.getUnSequenceFileList().size());
+    assertEquals(1, processor.getUnSequenceFileList().size());

Review comment:
       Because the logic of judging whether the file exists according to the plan index has changed, the file is not considered to exist when it is equal to. So in the test `testInstallPartial`,  there will be one file to be judged as an unseq file.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40210606/badge)](https://coveralls.io/builds/40210606)
   
   Coverage increased (+0.02%) to 67.928% when pulling **57cdff53c4b77bdbf2289c39fa478f605a6f28a9 on cluster_scalability** into **a99bf9c78eb02f2258f442ae7af725f3fff23ad3 on master**.
   


-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865107395






-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644885297



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
##########
@@ -55,22 +56,26 @@
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
    * @return the new group generated by the node

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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865495762


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='53.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [53.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644872189



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
##########
@@ -77,11 +115,17 @@ public boolean equals(Object o) {
       return false;
     }
     RemoveNodeLog that = (RemoveNodeLog) o;
-    return Objects.equals(removedNode, that.removedNode);
+    return Objects.equals(removedNode, that.removedNode)

Review comment:
       The answer is the same as above.




-- 
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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40340893/badge)](https://coveralls.io/builds/40340893)
   
   Coverage decreased (-0.09%) to 67.563% when pulling **ca0228d1f9c2f6c69150fbc153b42e870261ec39 on cluster_scalability** into **11c942a3b5bdcf046c334d79d65b0ba06e84107d on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r644925946



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -507,11 +541,16 @@ private SeriesReader getSeriesReader(
       Filter valueFilter,
       QueryContext context,
       Node header,
-      boolean ascending)
+      int raftId,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    if (requiredSlots == null) {

Review comment:
       I have added comments.
   
   > why don't judge this in other functions such as getMultSeriesReader?
   
   
   Because all other readers will call this function eventually.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-865495762


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [136 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='53.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [53.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [0.9% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r648496396



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
##########
@@ -304,6 +311,74 @@ private TSStatus forwardPlan(List<PartitionGroup> partitionGroups, PhysicalPlan
     return status;
   }
 
+  public void sendLogToAllDataGroups(Log log) throws ChangeMembershipException {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: start", log);
+    }
+
+    Map<PhysicalPlan, PartitionGroup> planGroupMap = router.splitAndRouteChangeMembershipLog(log);
+    List<String> errorCodePartitionGroups = new CopyOnWriteArrayList<>();
+    CountDownLatch counter = new CountDownLatch(planGroupMap.size());
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      metaGroupMember
+          .getAppendLogThreadPool()
+          .submit(() -> forwardChangeMembershipPlan(log, entry, errorCodePartitionGroups, counter));
+    }
+    try {
+      counter.await();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new ChangeMembershipException(
+          String.format("Can not wait all data groups to apply %s", log));
+    }
+    if (!errorCodePartitionGroups.isEmpty()) {
+      throw new ChangeMembershipException(
+          String.format("Apply %s failed with status {%s}", log, errorCodePartitionGroups));
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: end", log);
+    }
+  }
+
+  private void forwardChangeMembershipPlan(
+      Log log,
+      Map.Entry<PhysicalPlan, PartitionGroup> entry,
+      List<String> errorCodePartitionGroups,
+      CountDownLatch counter) {
+    int retryTime = 0;
+    try {
+      while (true) {
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "Send change membership log {} to data group {}, retry time: {}",
+              log,
+              entry.getValue(),
+              retryTime);
+        }
+        try {
+          TSStatus status = forwardToSingleGroup(entry);
+          if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+            if (logger.isDebugEnabled()) {
+              logger.debug(
+                  "Success to send change membership log {} to data group {}",
+                  log,
+                  entry.getValue());
+            }
+            return;
+          }
+          Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          errorCodePartitionGroups.add(e.getMessage());
+          return;
+        }
+        retryTime++;
+      }
+    } finally {
+      counter.countDown();
+    }
+  }

Review comment:
       Good suggestion. I have added timeout logic.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-866502119


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [138 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='52.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [52.9% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.2% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r639898089



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
##########
@@ -933,29 +1011,83 @@ public IBatchReader getSeriesBatchReader(
       Filter valueFilter,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots,
+      boolean syncLeader)
       throws StorageEngineException, QueryProcessException, IOException {
-    // pull the newest data
-    try {
-      dataGroupMember.syncLeaderWithConsistencyCheck(false);
-    } catch (CheckConsistencyException e) {
-      throw new StorageEngineException(e);
+    if (syncLeader) {
+      // pull the newest data
+      try {
+        dataGroupMember.syncLeaderWithConsistencyCheck(false);
+      } catch (CheckConsistencyException e) {
+        throw new StorageEngineException(e);
+      }
     }
 
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            timeFilter,
-            valueFilter,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
-    if (seriesReader.isEmpty()) {
-      return null;
+    // find the groups that should be queried due to data migration.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
+
+    // If requiredSlots is not null, it means that this data group is the previous holder of
+    // required slots, which is no need to merge other resource,
+    if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+      // merge remote reader and local reader
+      ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+
+      // add local reader
+      IPointReader seriesPointReader =
+          getSeriesPointReader(
+              path,
+              allSensors,
+              dataType,
+              timeFilter,
+              valueFilter,
+              context,
+              dataGroupMember,
+              ascending,
+              requiredSlots);
+      mergeReader.addReader(seriesPointReader, 0);

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.

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852279818


   
   [![Coverage Status](https://coveralls.io/builds/40750972/badge)](https://coveralls.io/builds/40750972)
   
   Coverage increased (+0.02%) to 67.908% when pulling **745209130978029347a09ba27e64ca5f6bff7dfd on cluster_scalability** into **d8b4ce8e6fbc4e25a55c7244a9e86592b8487002 on master**.
   


-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r645568987



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {
+    TSStatus result;
+    if (entry.getValue().contains(thisNode)) {
+      // the query should be handled by a group the local node is in, handle it with in the group
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Execute {} in a local group of {}", entry.getKey(), entry.getValue().getHeader());
+      result =
+          getLocalDataMember(entry.getValue().getHeader(), entry.getValue().getId())
+              .executeNonQueryPlan(entry.getKey());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    } else {
+      // forward the query to the group that should handle it
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Forward {} to a remote group of {}", entry.getKey(), entry.getValue().getHeader());
+      result = forwardPlan(entry.getKey(), entry.getValue());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    }
+    return result;
+  }
+
+  /**
+   * forward each sub-plan to its corresponding data group, if some groups goes wrong, the error
+   * messages from each group will be compacted into one string.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGroupMap) {

Review comment:
       Same as above.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1411,6 +1435,291 @@ public TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
     return result;
   }
 
+  /**
+   * Forward plans to the DataGroupMember of one node in the corresponding group. Only when all
+   * nodes time out, will a TIME_OUT be returned.
+   *
+   * @param planGroupMap sub-plan -> belong data group pairs
+   */
+  public TSStatus forwardPlan(Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan plan) {
+    // the error codes from the groups that cannot execute the plan
+    TSStatus status;
+    if (planGroupMap.size() == 1) {
+      status = forwardToSingleGroup(planGroupMap.entrySet().iterator().next());
+    } else {
+      if (plan instanceof InsertTabletPlan || plan instanceof CreateMultiTimeSeriesPlan) {
+        // InsertTabletPlan and CreateMultiTimeSeriesPlan contains many rows, each will correspond
+        // to a TSStatus as its
+        // execution result, as the plan is split and the sub-plans may have interleaving ranges,
+        // we must assure that each TSStatus is placed to the right position
+        // e.g., an InsertTabletPlan contains 3 rows, row1 and row3 belong to NodeA and row2
+        // belongs to NodeB, when NodeA returns a success while NodeB returns a failure, the
+        // failure and success should be placed into proper positions in TSStatus.subStatus
+        status = forwardMultiSubPlan(planGroupMap, plan);
+      } else {
+        status = forwardToMultipleGroup(planGroupMap);
+      }
+    }
+    if (plan instanceof InsertPlan
+        && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+        && config.isEnableAutoCreateSchema()) {
+      TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
+      if (tmpStatus != null) {
+        status = tmpStatus;
+      }
+    }
+    logger.debug("{}: executed {} with answer {}", name, plan, status);
+    return status;
+  }
+
+  private TSStatus createTimeseriesForFailedInsertion(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+    // try to create timeseries
+    if (plan.getFailedMeasurements() != null) {
+      plan.getPlanFromFailed();
+    }
+    boolean hasCreate;
+    try {
+      hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
+    } catch (IllegalPathException | CheckConsistencyException e) {
+      return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    if (hasCreate) {
+      return forwardPlan(planGroupMap, plan);
+    } else {
+      logger.error("{}, Cannot auto create timeseries.", thisNode);
+    }
+    return null;
+  }
+
+  /**
+   * Forward each sub-plan to its belonging data group, and combine responses from the groups.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private TSStatus forwardMultiSubPlan(
+      Map<PhysicalPlan, PartitionGroup> planGroupMap, PhysicalPlan parentPlan) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    TSStatus[] subStatus = null;
+    boolean noFailure = true;
+    boolean isBatchFailure = false;
+    EndPoint endPoint = null;
+    int totalRowNum = 0;
+    // send sub-plans to each belonging data group and collect results
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
+      noFailure = (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
+      isBatchFailure =
+          (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) || isBatchFailure;
+      if (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+        if (parentPlan instanceof InsertTabletPlan) {
+          totalRowNum = ((InsertTabletPlan) parentPlan).getRowCount();
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getIndexes().size();
+        }
+        if (subStatus == null) {
+          subStatus = new TSStatus[totalRowNum];
+          Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+        }
+        // set the status from one group to the proper positions of the overall status
+        if (parentPlan instanceof InsertTabletPlan) {
+          PartitionUtils.reordering(
+              (InsertTabletPlan) entry.getKey(),
+              subStatus,
+              tmpStatus.subStatus.toArray(new TSStatus[] {}));
+        } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
+          CreateMultiTimeSeriesPlan subPlan = (CreateMultiTimeSeriesPlan) entry.getKey();
+          for (int i = 0; i < subPlan.getIndexes().size(); i++) {
+            subStatus[subPlan.getIndexes().get(i)] = tmpStatus.subStatus.get(i);
+          }
+        }
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s:%s]",
+                tmpStatus.getCode(),
+                entry.getValue().getHeader(),
+                tmpStatus.getMessage(),
+                tmpStatus.subStatus));
+      }
+      if (parentPlan instanceof InsertTabletPlan
+          && tmpStatus.isSetRedirectNode()
+          && ((InsertTabletPlan) entry.getKey()).getMaxTime()
+              == ((InsertTabletPlan) parentPlan).getMaxTime()) {
+        endPoint = tmpStatus.getRedirectNode();
+      }
+    }
+
+    if (parentPlan instanceof CreateMultiTimeSeriesPlan
+        && !((CreateMultiTimeSeriesPlan) parentPlan).getResults().isEmpty()) {
+      if (subStatus == null) {
+        subStatus = new TSStatus[totalRowNum];
+        Arrays.fill(subStatus, RpcUtils.SUCCESS_STATUS);
+      }
+      noFailure = false;
+      isBatchFailure = true;
+      for (Entry<Integer, TSStatus> integerTSStatusEntry :
+          ((CreateMultiTimeSeriesPlan) parentPlan).getResults().entrySet()) {
+        subStatus[integerTSStatusEntry.getKey()] = integerTSStatusEntry.getValue();
+      }
+    }
+    return concludeFinalStatus(
+        noFailure, endPoint, isBatchFailure, subStatus, errorCodePartitionGroups);
+  }
+
+  private TSStatus concludeFinalStatus(
+      boolean noFailure,
+      EndPoint endPoint,
+      boolean isBatchFailure,
+      TSStatus[] subStatus,
+      List<String> errorCodePartitionGroups) {
+    TSStatus status;
+    if (noFailure) {
+      status = StatusUtils.OK;
+      if (endPoint != null) {
+        status = StatusUtils.getStatus(status, endPoint);
+      }
+    } else if (isBatchFailure) {
+      status = RpcUtils.getStatus(Arrays.asList(subStatus));
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR,
+              MSG_MULTIPLE_ERROR + errorCodePartitionGroups.toString());
+    }
+    return status;
+  }
+
+  private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {
+    TSStatus result;
+    if (entry.getValue().contains(thisNode)) {
+      // the query should be handled by a group the local node is in, handle it with in the group
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Execute {} in a local group of {}", entry.getKey(), entry.getValue().getHeader());
+      result =
+          getLocalDataMember(entry.getValue().getHeader(), entry.getValue().getId())
+              .executeNonQueryPlan(entry.getKey());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_LOCAL_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    } else {
+      // forward the query to the group that should handle it
+      long startTime =
+          Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+              .getOperationStartTime();
+      logger.debug(
+          "Forward {} to a remote group of {}", entry.getKey(), entry.getValue().getHeader());
+      result = forwardPlan(entry.getKey(), entry.getValue());
+      Timer.Statistic.META_GROUP_MEMBER_EXECUTE_NON_QUERY_IN_REMOTE_GROUP
+          .calOperationCostTimeFromStart(startTime);
+    }
+    return result;
+  }
+
+  /**
+   * forward each sub-plan to its corresponding data group, if some groups goes wrong, the error
+   * messages from each group will be compacted into one string.
+   *
+   * @param planGroupMap sub-plan -> data group pairs
+   */
+  private TSStatus forwardToMultipleGroup(Map<PhysicalPlan, PartitionGroup> planGroupMap) {
+    List<String> errorCodePartitionGroups = new ArrayList<>();
+    TSStatus tmpStatus;
+    boolean allRedirect = true;
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      tmpStatus = forwardToSingleGroup(entry);
+      if (!tmpStatus.isSetRedirectNode()) {
+        allRedirect = false;
+      }
+      if (tmpStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        logger.error("Fail to send log {} to data group {}", entry.getKey(), entry.getValue());
+        // execution failed, record the error message
+        errorCodePartitionGroups.add(
+            String.format(
+                "[%s@%s:%s]",
+                tmpStatus.getCode(), entry.getValue().getHeader(), tmpStatus.getMessage()));
+      }
+    }
+    TSStatus status;
+    if (errorCodePartitionGroups.isEmpty()) {
+      if (allRedirect) {
+        status = new TSStatus();
+        status.setCode(TSStatusCode.NEED_REDIRECTION.getStatusCode());
+      } else {
+        status = StatusUtils.OK;
+      }
+    } else {
+      status =
+          StatusUtils.getStatus(
+              StatusUtils.EXECUTE_STATEMENT_ERROR, MSG_MULTIPLE_ERROR + errorCodePartitionGroups);
+    }
+    return status;
+  }
+
+  /**
+   * Forward a plan to the DataGroupMember of one node in the group. Only when all nodes time out,
+   * will a TIME_OUT be returned.
+   */
+  private TSStatus forwardPlan(PhysicalPlan plan, PartitionGroup group) {

Review comment:
       Same as above.




-- 
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.

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



[GitHub] [iotdb] fanhualta commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
fanhualta commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r640798834



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java
##########
@@ -298,7 +299,7 @@ private void testNormal(boolean requiresReadOnly)
             loadedFiles.get(i).getMaxPlanIndex(),
             loadedFiles.get(i).getTsFile().getAbsolutePath());
       }
-      assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
+      assertEquals(-1, loadedFiles.get(i).getMaxPlanIndex());

Review comment:
       PullSnapshotTask is used for data migration between multiple raft groups. CachupTask is used for catch-up in one raft group.




-- 
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.

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



[GitHub] [iotdb] jt2594838 commented on a change in pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#discussion_r641384775



##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshotTest.java
##########
@@ -301,6 +301,6 @@ public void testInstallPartial()
     for (int i = 0; i < 9; i++) {
       assertEquals(i, loadedFiles.get(i).getMaxPlanIndex());
     }
-    assertEquals(0, processor.getUnSequenceFileList().size());
+    assertEquals(1, processor.getUnSequenceFileList().size());

Review comment:
       So the new modifications have drifted the case to another case, better to supplement the previous case where all files are covered by the pulled file.




-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852290094


   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=coverage&view=list) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo.png' alt='No Duplication information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-866502119


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [138 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50.png' alt='52.9%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list) [52.9% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5.png' alt='3.2%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list) [3.2% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=new_duplicated_lines_density&view=list)
   
   


-- 
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.

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



[GitHub] [iotdb] LebronAl commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-842166665


   Hi, I think it's time for us to start adding cluster design documents to the docs now~


-- 
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.

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



[GitHub] [iotdb] LebronAl edited a comment on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-854297217






-- 
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.

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3191: New features of cluster scalability and multi-raft

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3191:
URL: https://github.com/apache/iotdb/pull/3191#issuecomment-852261626


   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3191&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=coverage&view=list) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo.png' alt='No Duplication information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3191&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

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