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/07/08 08:41:58 UTC

[GitHub] [iotdb] neuyilan commented on a change in pull request #3269: [IOTDB-1292] Versioned MTree

neuyilan commented on a change in pull request #3269:
URL: https://github.com/apache/iotdb/pull/3269#discussion_r665847966



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
##########
@@ -1816,4 +1858,63 @@ public PartialPath getStorageGroupPath(PartialPath path) throws StorageGroupNotS
       return super.getStorageGroupPath(path);
     }
   }
+
+  @Override
+  public void createAlignedTimeSeries(CreateAlignedTimeSeriesPlan plan) throws MetadataException {
+    PartialPath path = plan.getPrefixPath();
+    if (checkSgNodeAndPlanMajorVersion(path)) {
+      super.createAlignedTimeSeries(plan);
+    }
+  }
+
+  @Override
+  public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException {
+    PartialPath path = plan.getPath();
+    if (checkSgNodeAndPlanMajorVersion(path)) {
+      super.createTimeseries(plan);
+    }
+  }
+
+  /**
+   * @param path the partial path
+   * @return true if the sg node major version == the path's major version; false if the sg node
+   *     major version > the path's major version; throw MetadataException if the sg node major
+   *     version < the path's major version.
+   * @throws MetadataException sync meta leader failed

Review comment:
       sure, I have amend the comment

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
##########
@@ -44,6 +44,10 @@
   // alias of measurement, null pointer cannot be serialized in thrift so empty string is instead
   protected String measurementAlias = "";
 
+  private long majorVersion = 0;
+
+  private long minorVersion = 0;
+

Review comment:
       done
   

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterUtils.java
##########
@@ -353,4 +361,65 @@ public static PartitionGroup partitionByPathTimeWithSync(
     }
     return partitionGroup;
   }
+
+  public static void setVersionForSpecialPlan(PhysicalPlan plan, long currLogIndex)
+      throws MetadataException {
+    switch (plan.getOperatorType()) {
+      case SET_STORAGE_GROUP:
+        plan.setMajorVersion(currLogIndex);
+        plan.setMinorVersion(0);
+        break;
+      case DELETE_STORAGE_GROUP:
+        List<PartialPath> deletePathList = new ArrayList<>();
+        for (PartialPath storageGroupPath : plan.getPaths()) {
+          List<PartialPath> allRelatedStorageGroupPath =
+              IoTDB.metaManager.getStorageGroupPaths(storageGroupPath);
+          if (allRelatedStorageGroupPath.isEmpty()) {
+            throw new PathNotExistException(storageGroupPath.getFullPath(), true);
+          }
+          for (PartialPath path : allRelatedStorageGroupPath) {
+            StorageGroupMNode storageGroupMNode = IoTDB.metaManager.getStorageGroupNodeByPath(path);
+            path.setMajorVersion(storageGroupMNode.getMajorVersion());
+            path.setMinorVersion(storageGroupMNode.getMinorVersion());
+          }
+        }
+        // replace the to be deleted paths
+        plan.setPaths(deletePathList);
+        break;

Review comment:
       Sure, we can just set the plan's major version and can delete the sg as long as its version is smaller than that of the plan.
   But, I want to unify the processing logic in the codeļ¼šthat is when applying one plan, only the plan's major version equals the node's major version can do it. I think this will make the code easier to understand. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
##########
@@ -974,9 +977,17 @@ TSStatus processPlanLocally(PhysicalPlan plan) {
     synchronized (logManager) {
       log.setCurrLogTerm(getTerm().get());
       log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
-
       log.setPlan(plan);
       plan.setIndex(log.getCurrLogIndex());
+
+      // Set version for some special plans
+      try {
+        ClusterUtils.setVersionForSpecialPlan(plan, log.getCurrLogIndex());
+      } catch (MetadataException e) {
+        logger.error("process plan failed, plan={}", plan, e);
+        return StatusUtils.INTERNAL_ERROR;

Review comment:
       done

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
##########
@@ -437,7 +448,7 @@ public static PhysicalPlan convertFromString(String str) {
                 CompressionType.values()[Integer.parseInt(words[5])]));
       case "1":
         return new StorageGroupMNodePlan(
-            words[1], Long.parseLong(words[2]), Integer.parseInt(words[3]));
+            words[1], Long.parseLong(words[2]), Integer.parseInt(words[3]), 0, 0);
       case "0":

Review comment:
       This function is only used for upgrade, in the previous version(<0.13), the node has no versions. so there I use zero.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
##########
@@ -166,6 +166,14 @@ public void setStorageGroup(String storageGroup) throws IOException {
     lineNumber.incrementAndGet();
   }
 
+  public void updateStorageGroup(String storageGroup) throws IOException {
+    String outputStr =
+        MetadataOperationType.UPDATE_STORAGE_GROUP + "," + storageGroup + LINE_SEPARATOR;
+    ByteBuffer buff = ByteBuffer.wrap(outputStr.getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+

Review comment:
       Sure, we have serialized the versions and after a restart, the plan can be deserialized correctly, the above codes is just one parse util that parses mlog.bin to mlog.txt




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