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 2020/11/24 13:03:10 UTC

[GitHub] [iotdb] samperson1997 commented on a change in pull request #1721: [IOTDB-868] fix comma bug for mlog

samperson1997 commented on a change in pull request #1721:
URL: https://github.com/apache/iotdb/pull/1721#discussion_r529510601



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.metadata.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      System.out.println("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Parse error: " + e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    ConsoleReader reader = new ConsoleReader();
+    reader.setExpandEvents(false);
+    try {
+      parseBasicParams(commandLine, reader);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      System.out.println("Encounter an error, because: " + e.getMessage());
+    } finally {
+      reader.close();
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine, ConsoleReader reader) throws Exception {

Review comment:
       `ConsoleReader reader` is not used in this method : )

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.metadata.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      System.out.println("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Parse error: " + e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    ConsoleReader reader = new ConsoleReader();
+    reader.setExpandEvents(false);
+    try {
+      parseBasicParams(commandLine, reader);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      System.out.println("Encounter an error, because: " + e.getMessage());
+    } finally {
+      reader.close();
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine, ConsoleReader reader) throws Exception {
+    inputFile = checkRequiredArg(FILE_ARGS, FILE_NAME, commandLine);
+    outputFile = commandLine.getOptionValue(OUT_ARGS);
+
+    if (outputFile == null) {
+      outputFile = "tmp.txt";
+    }
+  }
+
+  public static String checkRequiredArg(String arg, String name, CommandLine commandLine)
+    throws Exception {
+    String str = commandLine.getOptionValue(arg);
+    if (str == null) {
+      String msg = String.format("Required values for option '%s' not provided", name);
+      System.out.println(msg);
+      System.out.println("Use -help for more information");
+      throw new Exception(msg);
+    }
+    return str;
+  }
+
+  public static void parseFromFile(String inputFile, String outputFile) throws IOException {
+    try (MLogReader mLogReader = new MLogReader(inputFile);
+         MLogTxtWriter mLogTxtWriter = new MLogTxtWriter(outputFile)) {
+
+      while (mLogReader.hasNext()) {
+        PhysicalPlan plan = mLogReader.next();
+        switch (plan.getOperatorType()) {
+          case CREATE_TIMESERIES:
+            mLogTxtWriter.createTimeseries((CreateTimeSeriesPlan)plan,
+              ((CreateTimeSeriesPlan) plan).getTagOffset());
+            break;
+          case DELETE_TIMESERIES: {
+            List<PartialPath> pathList = plan.getPaths();
+            for (PartialPath partialPath : pathList) {
+              mLogTxtWriter.deleteTimeseries(partialPath.getFullPath());
+            }
+          }

Review comment:
       It's not recommended to add braces `{}` here. Maybe we could extract `pathList ` outside the `switch` statement?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.metadata.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      System.out.println("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Parse error: " + e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    ConsoleReader reader = new ConsoleReader();
+    reader.setExpandEvents(false);
+    try {
+      parseBasicParams(commandLine, reader);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      System.out.println("Encounter an error, because: " + e.getMessage());
+    } finally {
+      reader.close();
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine, ConsoleReader reader) throws Exception {
+    inputFile = checkRequiredArg(FILE_ARGS, FILE_NAME, commandLine);
+    outputFile = commandLine.getOptionValue(OUT_ARGS);
+
+    if (outputFile == null) {
+      outputFile = "tmp.txt";
+    }
+  }
+
+  public static String checkRequiredArg(String arg, String name, CommandLine commandLine)
+    throws Exception {

Review comment:
       ... and this `Exception` could be more specific as well

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;

Review comment:
       Remove unused import

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.metadata.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      System.out.println("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Parse error: " + e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    ConsoleReader reader = new ConsoleReader();
+    reader.setExpandEvents(false);
+    try {
+      parseBasicParams(commandLine, reader);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      System.out.println("Encounter an error, because: " + e.getMessage());
+    } finally {
+      reader.close();
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine, ConsoleReader reader) throws Exception {
+    inputFile = checkRequiredArg(FILE_ARGS, FILE_NAME, commandLine);
+    outputFile = commandLine.getOptionValue(OUT_ARGS);
+
+    if (outputFile == null) {
+      outputFile = "tmp.txt";
+    }
+  }
+
+  public static String checkRequiredArg(String arg, String name, CommandLine commandLine)
+    throws Exception {
+    String str = commandLine.getOptionValue(arg);
+    if (str == null) {
+      String msg = String.format("Required values for option '%s' not provided", name);
+      System.out.println(msg);
+      System.out.println("Use -help for more information");
+      throw new Exception(msg);

Review comment:
       I suggest to define and throw a specific exception. What do you think?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1262,9 +1214,68 @@ private void findNodes(MNode node, PartialPath path, List<PartialPath> res, int
   }
 
   public void serializeTo(String snapshotPath) throws IOException {
-    try (BufferedWriter bw = new BufferedWriter(
-        new FileWriter(SystemFileFactory.INSTANCE.getFile(snapshotPath)))) {
-      root.serializeTo(bw);
+    try (MLogWriter mLogWriter = new MLogWriter(snapshotPath)) {
+      root.serializeTo(mLogWriter);
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  public static MTree deserializeFrom(File mtreeSnapshot) {
+
+    try (MLogReader mlogReader = new MLogReader(mtreeSnapshot)) {
+      Deque<MNode> nodeStack = new ArrayDeque<>();
+      MNode node = null;
+
+      while (mlogReader.hasNext()) {
+        PhysicalPlan plan = null;
+        try {
+          plan = mlogReader.next();
+          if (plan == null) {
+            continue;
+          }
+          int childrenSize = 0;
+          if (plan instanceof StorageGroupMNodePlan) {
+            node = StorageGroupMNode.deserializeFrom((StorageGroupMNodePlan) plan);
+            childrenSize = ((StorageGroupMNodePlan) plan).getChildSize();
+          } else if (plan instanceof MeasurementMNodePlan) {
+            node = MeasurementMNode.deserializeFrom((MeasurementMNodePlan) plan);
+            childrenSize = ((MeasurementMNodePlan) plan).getChildSize();
+          } else if (plan instanceof MNodePlan) {
+            node = new MNode(null, ((MNodePlan) plan).getName());
+            childrenSize = ((MNodePlan) plan).getChildSize();
+          }
+
+          if (childrenSize == 0) {
+            nodeStack.push(node);
+          } else {
+            ConcurrentHashMap<String, MNode> childrenMap = new ConcurrentHashMap<>();
+            for (int i = 0; i < childrenSize; i++) {
+              MNode child = nodeStack.removeFirst();
+              child.setParent(node);
+              childrenMap.put(child.getName(), child);
+              if (child instanceof MeasurementMNode) {
+                String alias = ((MeasurementMNode) child).getAlias();
+                if (alias != null) {
+                  node.addAlias(alias, child);
+                }
+              }
+            }
+            node.setChildren(childrenMap);
+            nodeStack.push(node);
+          }
+        } catch (Exception e) {

Review comment:
       Maybe this `Exception` could be more specific too?

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -184,51 +185,16 @@ public void checkConfig() throws IOException {
             inputStream, TSFileConfig.STRING_CHARSET)) {
       properties.load(inputStreamReader);
     }
-    // check whether upgrading from v0.9 to v0.11

Review comment:
       Do you think it is necessary to delete all these codes (`upgradePropertiesFile`, `checkUnClosedTsFileV2`, `checkUnClosedTsFileV2InFolders`) now? I think this check should be modified before 0.12 is released... Or the users who are using v0.9 and intending to upgrade to master may encounter problems ... @qiaojialin 

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.metadata.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      System.out.println("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Parse error: " + e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    ConsoleReader reader = new ConsoleReader();
+    reader.setExpandEvents(false);
+    try {
+      parseBasicParams(commandLine, reader);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      System.out.println("Encounter an error, because: " + e.getMessage());
+    } finally {
+      reader.close();
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine, ConsoleReader reader) throws Exception {
+    inputFile = checkRequiredArg(FILE_ARGS, FILE_NAME, commandLine);
+    outputFile = commandLine.getOptionValue(OUT_ARGS);
+
+    if (outputFile == null) {
+      outputFile = "tmp.txt";
+    }
+  }
+
+  public static String checkRequiredArg(String arg, String name, CommandLine commandLine)
+    throws Exception {
+    String str = commandLine.getOptionValue(arg);
+    if (str == null) {
+      String msg = String.format("Required values for option '%s' not provided", name);
+      System.out.println(msg);
+      System.out.println("Use -help for more information");
+      throw new Exception(msg);
+    }
+    return str;
+  }
+
+  public static void parseFromFile(String inputFile, String outputFile) throws IOException {
+    try (MLogReader mLogReader = new MLogReader(inputFile);
+         MLogTxtWriter mLogTxtWriter = new MLogTxtWriter(outputFile)) {
+
+      while (mLogReader.hasNext()) {
+        PhysicalPlan plan = mLogReader.next();
+        switch (plan.getOperatorType()) {
+          case CREATE_TIMESERIES:
+            mLogTxtWriter.createTimeseries((CreateTimeSeriesPlan)plan,
+              ((CreateTimeSeriesPlan) plan).getTagOffset());
+            break;
+          case DELETE_TIMESERIES: {
+            List<PartialPath> pathList = plan.getPaths();
+            for (PartialPath partialPath : pathList) {
+              mLogTxtWriter.deleteTimeseries(partialPath.getFullPath());
+            }
+          }
+            break;
+          case SET_STORAGE_GROUP:
+            mLogTxtWriter.setStorageGroup(((SetStorageGroupPlan) plan).getPath().getFullPath());
+            break;
+          case DELETE_STORAGE_GROUP: {
+            List<PartialPath> pathList = plan.getPaths();
+            for (PartialPath partialPath : pathList) {
+              mLogTxtWriter.deleteStorageGroup(partialPath.getFullPath());
+            }
+          }

Review comment:
       ... so that braces `{}` could also be omitted here

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1262,9 +1214,68 @@ private void findNodes(MNode node, PartialPath path, List<PartialPath> res, int
   }
 
   public void serializeTo(String snapshotPath) throws IOException {
-    try (BufferedWriter bw = new BufferedWriter(
-        new FileWriter(SystemFileFactory.INSTANCE.getFile(snapshotPath)))) {
-      root.serializeTo(bw);
+    try (MLogWriter mLogWriter = new MLogWriter(snapshotPath)) {
+      root.serializeTo(mLogWriter);
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  public static MTree deserializeFrom(File mtreeSnapshot) {
+
+    try (MLogReader mlogReader = new MLogReader(mtreeSnapshot)) {
+      Deque<MNode> nodeStack = new ArrayDeque<>();
+      MNode node = null;
+
+      while (mlogReader.hasNext()) {
+        PhysicalPlan plan = null;
+        try {
+          plan = mlogReader.next();
+          if (plan == null) {
+            continue;
+          }
+          int childrenSize = 0;
+          if (plan instanceof StorageGroupMNodePlan) {
+            node = StorageGroupMNode.deserializeFrom((StorageGroupMNodePlan) plan);
+            childrenSize = ((StorageGroupMNodePlan) plan).getChildSize();
+          } else if (plan instanceof MeasurementMNodePlan) {
+            node = MeasurementMNode.deserializeFrom((MeasurementMNodePlan) plan);
+            childrenSize = ((MeasurementMNodePlan) plan).getChildSize();
+          } else if (plan instanceof MNodePlan) {
+            node = new MNode(null, ((MNodePlan) plan).getName());
+            childrenSize = ((MNodePlan) plan).getChildSize();
+          }
+
+          if (childrenSize == 0) {
+            nodeStack.push(node);
+          } else {
+            ConcurrentHashMap<String, MNode> childrenMap = new ConcurrentHashMap<>();
+            for (int i = 0; i < childrenSize; i++) {
+              MNode child = nodeStack.removeFirst();
+              child.setParent(node);
+              childrenMap.put(child.getName(), child);
+              if (child instanceof MeasurementMNode) {
+                String alias = ((MeasurementMNode) child).getAlias();
+                if (alias != null) {
+                  node.addAlias(alias, child);
+                }
+              }
+            }
+            node.setChildren(childrenMap);
+            nodeStack.push(node);
+          }

Review comment:
       Refactor as below may be more logical:
   ```
           if (childrenSize != 0) {
               ConcurrentHashMap<String, MNode> childrenMap = new ConcurrentHashMap<>();
               for (int i = 0; i < childrenSize; i++) {
                 MNode child = nodeStack.removeFirst();
                 child.setParent(node);
                 childrenMap.put(child.getName(), child);
                 if (child instanceof MeasurementMNode) {
                   String alias = ((MeasurementMNode) child).getAlias();
                   if (alias != null) {
                     node.addAlias(alias, child);
                   }
                 }
               }
               node.setChildren(childrenMap);
             }
             nodeStack.push(node);
           }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtReader.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.logfile;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+
+/**
+ * reader for reading mlog.txt
+ */
+public class MLogTxtReader implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogReader.class);

Review comment:
       ```suggestion
     private static final Logger logger = LoggerFactory.getLogger(MLogTxtReader.class);
   ```




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