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/09/11 02:35:29 UTC

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

HTHou commented on a change in pull request #1721:
URL: https://github.com/apache/incubator-iotdb/pull/1721#discussion_r486726741



##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -203,6 +203,13 @@ public void checkConfig() throws IOException {
       }
       // rename tmpLogFile to mlog
       FileUtils.moveFile(tmpMLogFile, mlogFile);
+
+      File oldMLogFile = SystemFileFactory.INSTANCE.getFile(SCHEMA_DIR + File.separator
+          + MetadataConstant.METADATA_OLD_LOG);
+
+      if (oldMLogFile.delete()) {
+        throw new IOException("Deleting old mlog " + oldMLogFile + "failed.");
+      }

Review comment:
       Should be `!oldMLogFile.delete()` here?

##########
File path: docs/SystemDesign/SchemaManager/SchemaManager.md
##########
@@ -213,7 +213,7 @@ The method is `MManager.initFromLog()`:
 
 ## Log management of metadata
 
-* org.apache.iotdb.db.metadata.MLogWriter
+* org.apache.iotdb.db.metadata.logfile.MLogWriter
 
 All metadata operations are recorded in a metadata log file, which defaults to data/system/schema/mlog.txt.

Review comment:
       ```suggestion
   All metadata operations are recorded in a metadata log file, which defaults to data/system/schema/mlog.bin.
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1002,55 +1008,74 @@ 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);
+    MLogWriter mLogWriter = null;
+    try {
+      mLogWriter = new MLogWriter(snapshotPath);
+      root.serializeTo(mLogWriter);
+    } finally {
+      if (mLogWriter != null) {
+        mLogWriter.close();
+      }
     }

Review comment:
       Use try with resource

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1002,55 +1008,74 @@ 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);
+    MLogWriter mLogWriter = null;
+    try {
+      mLogWriter = new MLogWriter(snapshotPath);
+      root.serializeTo(mLogWriter);
+    } finally {
+      if (mLogWriter != null) {
+        mLogWriter.close();
+      }
     }
   }
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public static MTree deserializeFrom(File mtreeSnapshot) {
-    try (BufferedReader br = new BufferedReader(new FileReader(mtreeSnapshot))) {
-      String s;
+    MLogReader mlogReader = null;
+    try {

Review comment:
       Use try with resource to avoid new sonar code smell..

##########
File path: docs/zh/SystemDesign/SchemaManager/SchemaManager.md
##########
@@ -211,7 +211,7 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
 
 ## 元数据日志管理
 
-* org.apache.iotdb.db.metadata.MLogWriter
+* org.apache.iotdb.db.metadata.logfile.MLogWriter
 
 所有元数据的操作均会记录到元数据日志文件中,此文件默认为 data/system/schema/mlog.txt。

Review comment:
       ```suggestion
   所有元数据的操作均会记录到元数据日志文件中,此文件默认为 data/system/schema/mlog.bin。
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1002,55 +1008,74 @@ 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);
+    MLogWriter mLogWriter = null;
+    try {
+      mLogWriter = new MLogWriter(snapshotPath);
+      root.serializeTo(mLogWriter);
+    } finally {
+      if (mLogWriter != null) {
+        mLogWriter.close();
+      }
     }
   }
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public static MTree deserializeFrom(File mtreeSnapshot) {
-    try (BufferedReader br = new BufferedReader(new FileReader(mtreeSnapshot))) {
-      String s;
+    MLogReader mlogReader = null;
+    try {
+      mlogReader = new MLogReader(mtreeSnapshot);
       Deque<MNode> nodeStack = new ArrayDeque<>();
       MNode node = null;
 
-      while ((s = br.readLine()) != null) {
-        String[] nodeInfo = s.split(",");
-        short nodeType = Short.parseShort(nodeInfo[0]);
-        if (nodeType == MetadataConstant.STORAGE_GROUP_MNODE_TYPE) {
-          node = StorageGroupMNode.deserializeFrom(nodeInfo);
-        } else if (nodeType == MetadataConstant.MEASUREMENT_MNODE_TYPE) {
-          node = MeasurementMNode.deserializeFrom(nodeInfo);
-        } else {
-          node = new MNode(null, nodeInfo[1]);
-        }
+      while (mlogReader.hasNext()) {
+        PhysicalPlan plan = null;
+        try {
+          plan = mlogReader.next();

Review comment:
       Use try with resource

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -248,21 +237,32 @@ private int initFromLog(File logFile) throws IOException {
     // init the metadata from the operation log
     if (logFile.exists()) {
       int idx = 0;
-      try (FileReader fr = new FileReader(logFile);
-          BufferedReader br = new BufferedReader(fr)) {
-        String cmd;
-        while ((cmd = br.readLine()) != null) {
+      MLogReader mLogReader = null;
+      try {
+        mLogReader = new MLogReader(config.getSchemaDir(), MetadataConstant.METADATA_LOG);

Review comment:
       Use try with resource

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.io.*;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.*;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.MetadataOperationType;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MLogWriter {
+
+  private static final Logger logger = LoggerFactory.getLogger(MLogWriter.class);
+  private File logFile;
+  private LogWriter logWriter;
+  private int logNum;
+  private ByteBuffer mlogBuffer = ByteBuffer.allocate(
+    IoTDBDescriptor.getInstance().getConfig().getMlogBufferSize());
+
+  public MLogWriter(String schemaDir, String logFileName) throws IOException {
+    File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
+    if (!metadataDir.exists()) {
+      if (metadataDir.mkdirs()) {
+        logger.info("create schema folder {}.", metadataDir);
+      } else {
+        logger.info("create schema folder {} failed.", metadataDir);

Review comment:
       info -> warn?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.io.*;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.*;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.MetadataOperationType;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MLogWriter {
+
+  private static final Logger logger = LoggerFactory.getLogger(MLogWriter.class);
+  private File logFile;
+  private LogWriter logWriter;
+  private int logNum;
+  private ByteBuffer mlogBuffer = ByteBuffer.allocate(
+    IoTDBDescriptor.getInstance().getConfig().getMlogBufferSize());
+
+  public MLogWriter(String schemaDir, String logFileName) throws IOException {
+    File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
+    if (!metadataDir.exists()) {
+      if (metadataDir.mkdirs()) {
+        logger.info("create schema folder {}.", metadataDir);
+      } else {
+        logger.info("create schema folder {} failed.", metadataDir);
+      }
+    }
+
+    logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + logFileName);
+    // always flush
+    logWriter = new LogWriter(logFile, 0L);
+  }
+
+  public MLogWriter(String logFilePath) throws IOException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+    // always flush
+    logWriter = new LogWriter(logFile, 0L);
+  }
+
+  public void close() throws IOException {
+    sync();
+    logWriter.close();
+  }
+
+  private void sync() {
+    try {
+      logWriter.write(mlogBuffer);
+    } catch (IOException e) {
+      logger.error("MLog {} sync failed, change system mode to read-only", logFile.getAbsoluteFile(), e);
+      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
+    }
+    mlogBuffer.clear();
+  }
+
+  private void putLog(PhysicalPlan plan) {
+    mlogBuffer.mark();
+    try {
+      plan.serialize(mlogBuffer);
+    } catch (BufferOverflowException e) {
+      logger.error("MLog {} BufferOverflow !", plan.getOperatorType(), e);
+      mlogBuffer.reset();
+      sync();
+      plan.serialize(mlogBuffer);
+    }
+    logNum ++;
+  }
+
+  public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws IOException {
+    try {
+      putLog(plan);
+      ChangeTagOffsetPlan changeTagOffsetPlan = new ChangeTagOffsetPlan(plan.getPath(), offset);
+      putLog(changeTagOffsetPlan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void deleteTimeseries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) throws IOException {
+    try {
+      putLog(deleteTimeSeriesPlan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void setStorageGroup(PartialPath storageGroup) throws IOException {
+    try {
+      SetStorageGroupPlan plan = new SetStorageGroupPlan(storageGroup);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void deleteStorageGroup(PartialPath storageGroup) throws IOException {
+    try {
+      DeleteStorageGroupPlan plan = new DeleteStorageGroupPlan(Collections.singletonList(storageGroup));
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void setTTL(PartialPath storageGroup, long ttl) throws IOException {
+    try {
+      SetTTLPlan plan = new SetTTLPlan(storageGroup, ttl);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void changeOffset(PartialPath path, long offset) throws IOException {
+    try {
+      ChangeTagOffsetPlan plan = new ChangeTagOffsetPlan(path, offset);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void changeAlias(PartialPath path, String alias) throws IOException {
+    try {
+      ChangeAliasPlan plan = new ChangeAliasPlan(path, alias);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void serializeMNode(MNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      MNodePlan plan = new MNodePlan(node.getName(), childSize);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void serializeMeasurementMNode(MeasurementMNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      MeasurementNodePlan plan = new MeasurementNodePlan(node.getName(), node.getAlias(),
+        node.getOffset(), childSize, node.getSchema());
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public void seriallizeStorageGroupMNode(StorageGroupMNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      StorageGroupMNodePlan plan = new StorageGroupMNodePlan(node.getName(), node.getDataTTL(), childSize);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        "Log cannot fit into buffer, please increase mlog_buffer_size", e);
+    }
+  }
+
+  public static void upgradeMLog(String schemaDir, String logFileName) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + logFileName);
+    File tmpLogFile = SystemFileFactory.INSTANCE.getFile(logFile.getAbsolutePath() + ".tmp");
+    File oldLogFile = SystemFileFactory.INSTANCE.getFile(
+        schemaDir + File.separator + MetadataConstant.METADATA_OLD_LOG);
+
+    if (oldLogFile.exists()) {
+      MLogWriter mLogWriter = null;
+      OldMLogReader oldMLogReader = null;
+      try {
+        // upgrade from old character log file to new binary mlog
+        mLogWriter = new MLogWriter(schemaDir, logFileName + ".tmp");
+        oldMLogReader = new OldMLogReader(schemaDir, MetadataConstant.METADATA_OLD_LOG);

Review comment:
       use try with resource

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -203,6 +203,13 @@ public void checkConfig() throws IOException {
       }
       // rename tmpLogFile to mlog
       FileUtils.moveFile(tmpMLogFile, mlogFile);
+
+      File oldMLogFile = SystemFileFactory.INSTANCE.getFile(SCHEMA_DIR + File.separator
+          + MetadataConstant.METADATA_OLD_LOG);

Review comment:
       Line 190 to line 205 is the part of upgrading v0.9 mlog to v0.10. Since the `MetadataConstant.METADATA_LOG` has changed, I don't think that upgrading will work correctly. Should we remove that?




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