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 2022/08/14 19:31:04 UTC

[GitHub] [iotdb] sogood99 opened a new pull request, #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

sogood99 opened a new pull request, #6989:
URL: https://github.com/apache/iotdb/pull/6989

   To reduce memory and disk usage, add in the functionality to separate out cold 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.

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r956523953


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  @Override
+  public void close() {
+    for (FileOutputStream logFileStream : logOutputMap.values()) {
+      if (logFileStream != null) {
+        try {
+          logFileStream.close();
+        } catch (IOException e) {
+          logger.error("log file could not be closed");
+        }
+      }
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /**
+   * started the migration task, write to
+   *
+   * @return true if write log successful, false otherwise
+   */
+  public boolean startTask(long taskId, File targetDir) throws IOException {
+    FileOutputStream logFileOutput;
+    if (logOutputMap.containsKey(taskId) && logOutputMap.get(taskId) != null) {
+      logOutputMap.get(taskId).close();
+      logOutputMap.remove(taskId);
+    }
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+    if (logFile.exists()) {
+      // want an empty log file
+      logFile.delete();
+    }
+    if (!logFile.createNewFile()) {
+      // log file doesn't exist but cannot be created
+      return false;
+    }
+
+    logFileOutput = new FileOutputStream(logFile);
+    logOutputMap.put(taskId, logFileOutput);
+
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+
+    return true;
+  }
+
+  /**
+   * started migrating tsfile and its resource/mod files
+   *
+   * @return true if write log successful, false otherwise
+   */
+  public boolean start(long taskId, File tsfile) throws IOException {
+    FileOutputStream logFileOutput;
+    if (logOutputMap.containsKey(taskId) && logOutputMap.get(taskId) != null) {
+      logFileOutput = logOutputMap.get(taskId);
+    } else {
+      File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+      if (!logFile.exists()) {
+        if (!logFile.createNewFile()) {
+          // log file doesn't exist but cannot be created
+          return false;
+        }
+      }
+
+      logFileOutput = new FileOutputStream(logFile);
+      logOutputMap.put(taskId, logFileOutput);
+    }
+
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(long taskId) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }
+    if (logOutputMap.containsKey(taskId)) {
+      try {
+        logOutputMap.get(taskId).close();
+      } catch (IOException e) {
+        logger.error("could not close fileoutputstream for task {}", taskId);
+      }
+      logOutputMap.remove(taskId);
+    }
+  }
+
+  /** finish the unfinished MigrationTasks using log files under MIGRATING_LOG_DIR */
+  public void recover() {
+    for (File logFile : MIGRATING_LOG_DIR.listFiles()) {
+      try {
+        FileInputStream logFileInput = new FileInputStream(logFile);
+        String targetDirPath = ReadWriteIOUtils.readString(logFileInput);
+
+        File targetDir = SystemFileFactory.INSTANCE.getFile(targetDirPath);
+        while (logFileInput.available() > 0) {

Review Comment:
   1. Fixed.
   2. finish(taskId) should delete it.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {

Review Comment:
   Renamed MigratingFileLogManager -> TsFileMigrationLogger, MigrationLogReader/Writer -> MigrationTaskReader/Writer.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950857565


##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -1254,6 +1257,46 @@ protected void getSeriesSchemas(InsertPlan insertPlan, VirtualStorageGroupProces
     }
   }
 
+  // push the migration info to migrationManager
+  public void setMigration(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    migrationManager.setMigrate(storageGroup, targetDir, ttl, startTime);
+    logger.info("start check migration task successfully.");
+  }
+
+  public void unsetMigration(long taskId, PartialPath storageGroup) {
+    if (taskId != -1) {

Review Comment:
   fixed



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }

Review Comment:
   fixed



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }
+    } catch (IOException e) {
+      logger.error("unable to write to migrate 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.

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962397246


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as pauseMigrate using taskId except does not write to log */
+  public boolean pauseMigrationFromLog(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      migrationTasks.get(pauseTaskId).setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Unpause migration task from migrationTasks list using taskId
+   *
+   * @param unpauseTaskId taskId of task to unpause
+   * @return true if task with index exists and paused
+   */
+  public boolean resumeMigration(long unpauseTaskId) {
+    if (migrationTasks.containsKey(unpauseTaskId)) {
+      MigrationTask task = migrationTasks.get(unpauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as pauseMigrate using taskId except does not write to log */
+  public boolean pauseMigrationFromLog(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      migrationTasks.get(pauseTaskId).setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Unpause migration task from migrationTasks list using taskId
+   *
+   * @param unpauseTaskId taskId of task to unpause
+   * @return true if task with index exists and paused
+   */
+  public boolean resumeMigration(long unpauseTaskId) {
+    if (migrationTasks.containsKey(unpauseTaskId)) {
+      MigrationTask task = migrationTasks.get(unpauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to READY
+        task.setStatus(MigrationTask.MigrationTaskStatus.READY);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Resume migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId and paused.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean resumeMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED)) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }

Review Comment:
   Moved to MigrationRecover.



-- 
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] qiaojialin merged pull request #6989: [To rel/0.13][IOTDB-3882] Support Data Archiving

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


-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950857585


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }
+    } catch (IOException e) {
+      logger.error("unable to write to migrate 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.

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950857578


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, 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.

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950857525


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;
+

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -1254,6 +1257,46 @@ protected void getSeriesSchemas(InsertPlan insertPlan, VirtualStorageGroupProces
     }
   }
 
+  // push the migration info to migrationManager
+  public void setMigration(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    migrationManager.setMigrate(storageGroup, targetDir, ttl, startTime);
+    logger.info("start check migration task successfully.");
+  }
+
+  public void unsetMigration(long taskId, PartialPath storageGroup) {
+    if (taskId != -1) {

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r954418165


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }
+  }
+
+  /** finish the unfinished MigrationTasks using log files under MIGRATING_LOG_DIR */
+  public void recover() throws IOException {
+    for (File logFile : MIGRATING_LOG_DIR.listFiles()) {
+      FileInputStream logFileInput = new FileInputStream(logFile);
+
+      String tsfilePath = ReadWriteIOUtils.readString(logFileInput);
+      String targetDirPath = ReadWriteIOUtils.readString(logFileInput);
+
+      File tsfile = SystemFileFactory.INSTANCE.getFile(tsfilePath);
+      File targetDir = SystemFileFactory.INSTANCE.getFile(targetDirPath);
+
+      if (targetDir.exists()) {
+        if (!targetDir.isDirectory()) {
+          logger.error("target dir {} not a directory", targetDirPath);
+          return;
+        }
+      } else if (!targetDir.mkdirs()) {
+        logger.error("create target dir {} failed", targetDirPath);
+        return;
+      }
+
+      TsFileResource resource = new TsFileResource(tsfile);
+      resource.migrate(targetDir);
+      finish(tsfile);
+    }
+  }

Review Comment:
   1. Added checks for File.exist() in TsFileResource.migrate so it should be fine here.
   2. Fixed.
   3. Added.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Support Data Archiving

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r976656357


##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration
+setMigration
+    : SET MIGRATION TO storageGroup=prefixPath startTime=DATETIME_LITERAL ttl=INTEGER_LITERAL targetDir=STRING_LITERAL
+    | SET MIGRATION TO setMigrationClause*
+    ;
+
+setMigrationClause
+    : STORAGE_GROUP     OPERATOR_EQ storageGroup=prefixPath
+    | START_TIME        OPERATOR_EQ startTime=DATETIME_LITERAL
+    | TTL               OPERATOR_EQ ttl=INTEGER_LITERAL
+    | TARGET_DIR        OPERATOR_EQ targetDir=STRING_LITERAL
+    ;
+
+// Cancel Migration
+cancelMigration
+    : CANCEL MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Pause Migration
+pauseMigration
+    : PAUSE MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Unpause/Resume migration
+resumeMigration
+    : RESUME MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Show Migration
+showMigration
+    : SHOW MIGRATION ON prefixPath (COMMA prefixPath)*

Review Comment:
   Changed.



##########
integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMigrationIT.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.integration;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.utils.FileUtils;
+import org.apache.iotdb.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBMigrationIT {

Review Comment:
   Changed.



##########
server/src/assembly/resources/conf/iotdb-engine.properties:
##########
@@ -275,6 +275,10 @@ timestamp_precision=ms
 # Datatype: int
 # concurrent_query_thread=16
 
+# How many threads can concurrently run migration tasks. When <= 0, use CPU core number.
+# Datatype: int
+# migration_thread_num=2

Review Comment:
   Changed.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r952074748


##########
server/src/assembly/resources/conf/iotdb-engine.properties:
##########
@@ -275,6 +275,10 @@ timestamp_precision=ms
 # Datatype: int
 # concurrent_query_thread=16
 
+# How many threads can concurrently run migration tasks. When <= 0, use CPU core number.
+# Datatype: int
+# migration_thread_num=16

Review Comment:
   The migration tasks are not performance-critical, using 16 threads to execute this task is unsuitable.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationLogWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationLogWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationLogWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationLogReader logReader = new MigrationLogReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case UNSET:
+            unsetMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case UNPAUSE:
+            unpauseMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    // finished migrating the tsfiles that were in process
+    try {
+      MigratingFileLogManager.getInstance().recover();
+    } catch (IOException e) {
+      logger.error("migratingfile could not recover");
+    }

Review Comment:
   You should recover before the system is ready, please reference the code `registerManager.register(StorageEngine.getInstance());` in the IoTDB class.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }
+  }
+
+  /** finish the unfinished MigrationTasks using log files under MIGRATING_LOG_DIR */
+  public void recover() throws IOException {
+    for (File logFile : MIGRATING_LOG_DIR.listFiles()) {
+      FileInputStream logFileInput = new FileInputStream(logFile);
+
+      String tsfilePath = ReadWriteIOUtils.readString(logFileInput);
+      String targetDirPath = ReadWriteIOUtils.readString(logFileInput);
+
+      File tsfile = SystemFileFactory.INSTANCE.getFile(tsfilePath);
+      File targetDir = SystemFileFactory.INSTANCE.getFile(targetDirPath);
+
+      if (targetDir.exists()) {
+        if (!targetDir.isDirectory()) {
+          logger.error("target dir {} not a directory", targetDirPath);
+          return;
+        }
+      } else if (!targetDir.mkdirs()) {
+        logger.error("create target dir {} failed", targetDirPath);
+        return;
+      }
+
+      TsFileResource resource = new TsFileResource(tsfile);
+      resource.migrate(targetDir);
+      finish(tsfile);
+    }
+  }

Review Comment:
   There problems as below:
   1. It‘s not a good idea to use TsFileResource.migrate here because this method migrates .tsfile, .resource and .mods files at the same time. If any file is absent, this method will throw an eception.
   2. The for statement will exit when any error occurs in the loop, you should catch the exceptions, deal with errors and then continue the loop.
   3. Add some test to simulate some situations that .tsfile, .resource or .mods file is absent.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }

Review Comment:
   I have an doubt here, maybe you can test the performance between the following implementations:
   1. Just create one log file, and log all migration information into it.
   2. Create one log file for each migration task, and log its' migration information into it.
   3. Create one log file for each migration file (your current implementation). I think this implementation may slow because you submit a large amount of small I/O tasks.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private volatile long taskId;
+  private volatile PartialPath storageGroup;
+  private volatile File targetDir;
+  private volatile long startTime;
+  private volatile long ttl;
+  private volatile MigrationTaskStatus status = MigrationTaskStatus.READY;

Review Comment:
   Just add volatile keyword for status field is ok.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private volatile long taskId;
+  private volatile PartialPath storageGroup;
+  private volatile File targetDir;
+  private volatile long startTime;
+  private volatile long ttl;
+  private volatile MigrationTaskStatus status = MigrationTaskStatus.READY;
+
+  public MigrationTask(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    this.taskId = taskId;
+    this.storageGroup = storageGroup;
+    this.targetDir = targetDir;
+    this.ttl = ttl;
+    this.startTime = startTime;
+  }
+
+  // getter and setter functions
+
+  public long getTaskId() {
+    return taskId;
+  }
+
+  public PartialPath getStorageGroup() {
+    return storageGroup;
+  }
+
+  public File getTargetDir() {
+    return targetDir;
+  }
+
+  public long getTTL() {
+    return ttl;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public MigrationTaskStatus getStatus() {
+    return status;
+  }
+
+  public void setTaskId(long taskId) {
+    this.taskId = taskId;
+  }
+
+  public void setStorageGroup(PartialPath storageGroup) {
+    this.storageGroup = storageGroup;
+  }
+
+  public void setTargetDir(File targetDir) {
+    this.targetDir = targetDir;
+  }
+
+  public void setTTL(long ttl) {
+    this.ttl = ttl;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }

Review Comment:
   Delete the redundant set methods, it's dangerous to expose all fields to other classes.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r954418383


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private volatile long taskId;
+  private volatile PartialPath storageGroup;
+  private volatile File targetDir;
+  private volatile long startTime;
+  private volatile long ttl;
+  private volatile MigrationTaskStatus status = MigrationTaskStatus.READY;

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962397160


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;

Review Comment:
   To keep the taskId the same as before shutdown.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950862629


##########
server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java:
##########
@@ -760,6 +778,8 @@ protected QueryDataSet processShowQuery(ShowPlan showPlan, QueryContext context)
         return processShowPathsSetSchemaTemplate((ShowPathsSetTemplatePlan) showPlan);
       case PATHS_USING_SCHEMA_TEMPLATE:
         return processShowPathsUsingSchemaTemplate((ShowPathsUsingTemplatePlan) showPlan);
+      case MIGRATION:

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private long taskId;
+  private PartialPath storageGroup;
+  private File targetDir;
+  private long startTime;
+  private long ttl;
+  private MigrationTaskStatus status = MigrationTaskStatus.READY;

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950880099


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java:
##########
@@ -1555,6 +1556,72 @@ public void timedFlushSeqMemTable() {
     }
   }
 
+  /** iterate over TsFiles and migrate to targetDir if out of ttl */
+  public void checkMigration(MigrationTask task) {

Review Comment:
   Currently i dont think a way to access the all virtual storage group processor in StorageGroupManager.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r956850686


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, TsFileMigrationLogger writes files
+ * to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) , then
+ * deletes it after the task has finished operation.
+ */
+public class TsFileMigrationLogger implements AutoCloseable {

Review Comment:
   This class isn‘t AutoCloseable, please delete this interface.



##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -587,6 +591,8 @@ public void shutdown(long milliseconds) throws ShutdownException {
     shutdownTimedService(tsFileTimedCloseCheckThread, "TsFileTimedCloseCheckThread");
     recoveryThreadPool.shutdownNow();
     processorMap.clear();
+    migrationManager.shutdown();
+    TsFileMigrationLogger.getInstance().close();

Review Comment:
   Move this into MigrationManager.shutdown



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;

Review Comment:
   Please use your own logger and config. Never use other classes' loggers.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);

Review Comment:
   Please close this when the current system exits.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, TsFileMigrationLogger writes files
+ * to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) , then
+ * deletes it after the task has finished operation.
+ */
+public class TsFileMigrationLogger implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();

Review Comment:
   1. Please use map interface
   2. It's better for each migration task to control it's own resources, like FileOutputStream here. It‘s unsuitable for MigrationTask class to only stores some information, but does‘t take any responsibilities to actions. So, maybe you should move these responsibilities into MigrationTask itself and delete this class.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();

Review Comment:
   1. Please use 'try'-with-resources statement.
   2. First read log, then create logWriter.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r954418801


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationLogWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationLogWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationLogWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationLogReader logReader = new MigrationLogReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case UNSET:
+            unsetMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case UNPAUSE:
+            unpauseMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    // finished migrating the tsfiles that were in process
+    try {
+      MigratingFileLogManager.getInstance().recover();
+    } catch (IOException e) {
+      logger.error("migratingfile could not recover");
+    }

Review Comment:
   Moved to IoTDB.setUp.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r954419510


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private volatile long taskId;
+  private volatile PartialPath storageGroup;
+  private volatile File targetDir;
+  private volatile long startTime;
+  private volatile long ttl;
+  private volatile MigrationTaskStatus status = MigrationTaskStatus.READY;
+
+  public MigrationTask(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    this.taskId = taskId;
+    this.storageGroup = storageGroup;
+    this.targetDir = targetDir;
+    this.ttl = ttl;
+    this.startTime = startTime;
+  }
+
+  // getter and setter functions
+
+  public long getTaskId() {
+    return taskId;
+  }
+
+  public PartialPath getStorageGroup() {
+    return storageGroup;
+  }
+
+  public File getTargetDir() {
+    return targetDir;
+  }
+
+  public long getTTL() {
+    return ttl;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public MigrationTaskStatus getStatus() {
+    return status;
+  }
+
+  public void setTaskId(long taskId) {
+    this.taskId = taskId;
+  }
+
+  public void setStorageGroup(PartialPath storageGroup) {
+    this.storageGroup = storageGroup;
+  }
+
+  public void setTargetDir(File targetDir) {
+    this.targetDir = targetDir;
+  }
+
+  public void setTTL(long ttl) {
+    this.ttl = ttl;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = 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.

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r956523808


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;

Review Comment:
   Changed to 2.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962396293


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java:
##########
@@ -512,6 +512,15 @@ public void loadProperties(Properties properties) throws UnknownHostException {
       conf.setConcurrentSubRawQueryThread(Runtime.getRuntime().availableProcessors());
     }
 
+    conf.setMigrationThread(
+        Integer.parseInt(
+            properties.getProperty(
+                "migration_thread_num", Integer.toString(conf.getMigrationThread()))));
+
+    if (conf.getMigrationThread() <= 0) {
+      conf.setMigrationThread(Runtime.getRuntime().availableProcessors());
+    }

Review Comment:
   Changed.



##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -865,6 +865,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 2;

Review Comment:
   Changed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");

Review Comment:
   Changed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }

Review Comment:
   Changed.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962397308


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTaskWriter.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationTasKWriter writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationTaskWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationTaskWriter(String logFileName) throws FileNotFoundException {
+    this(SystemFileFactory.INSTANCE.getFile(logFileName));
+  }
+
+  public MigrationTaskWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }

Review Comment:
   Changed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTaskWriter.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationTasKWriter writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationTaskWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationTaskWriter(String logFileName) throws FileNotFoundException {
+    this(SystemFileFactory.INSTANCE.getFile(logFileName));
+  }
+
+  public MigrationTaskWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }
+
+      logFileOutStream.flush();
+    } catch (IOException e) {
+      logger.error("unable to write to migrate log");
+    }
+  }
+
+  public void setMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log =
+        new MigrationLog(
+            MigrationLog.LogType.SET,
+            migrationTask.getTaskId(),
+            migrationTask.getStorageGroup(),
+            migrationTask.getTargetDir().getPath(),
+            migrationTask.getStartTime(),
+            migrationTask.getTTL());
+    putLog(log);
+  }
+
+  public void startMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.START, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void finishMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.FINISHED, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void unsetMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.CANCEL, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void pauseMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.PAUSE, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void unpauseMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.RESUME, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void error(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.ERROR, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  @Override
+  public void close() throws Exception {
+    logFileOutStream.close();
+  }
+
+  public static class MigrationLog {
+    public LogType type;
+    public long taskId;
+    public PartialPath storageGroup;
+    public String targetDirPath;
+    public long startTime;
+    public long ttl;
+
+    public MigrationLog() {}
+
+    public MigrationLog(LogType type, long taskId) {
+      this.type = type;
+      this.taskId = taskId;
+    }
+
+    public MigrationLog(
+        LogType type,
+        long taskId,
+        PartialPath storageGroup,
+        String targetDirPath,
+        long startTime,
+        long ttl) {
+      this.type = type;
+      this.taskId = taskId;
+      this.storageGroup = storageGroup;
+      this.targetDirPath = targetDirPath;
+      this.startTime = startTime;
+      this.ttl = ttl;
+    }
+
+    public MigrationLog(LogType type, MigrationTask task) {
+      this.type = type;
+      this.taskId = task.getTaskId();
+      this.storageGroup = task.getStorageGroup();
+      this.targetDirPath = task.getTargetDir().getPath();
+      this.startTime = task.getStartTime();
+      this.ttl = task.getTTL();
+    }
+
+    public enum LogType {
+      SET,
+      CANCEL,
+      START,
+      PAUSE,
+      RESUME,
+      FINISHED,
+      ERROR
+    }

Review Comment:
   Changed.



##########
server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java:
##########
@@ -785,6 +787,126 @@ public Operator visitUnsetSchemaTemplate(IoTDBSqlParser.UnsetSchemaTemplateConte
     return operator;
   }
 
+  // Set Migration
+  @Override
+  public Operator visitSetMigration(IoTDBSqlParser.SetMigrationContext ctx) {
+    SetMigrationOperator operator = new SetMigrationOperator(SQLConstant.TOK_SET);
+
+    if (ctx.storageGroup != null) {
+      operator.setStorageGroup(parsePrefixPath(ctx.storageGroup));
+    }
+    if (ctx.ttl != null) {
+      operator.setTTL(Long.parseLong(ctx.ttl.getText()));
+    }
+    if (ctx.startTime != null) {
+      operator.setStartTime(parseDateFormat(ctx.startTime.getText()));
+    }
+    if (ctx.targetDir != null) {
+      FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+      File targetDir = fsFactory.getFile(parseStringLiteral(ctx.targetDir.getText()));
+      if (!targetDir.exists()) {
+        throw new SQLParserException("unknown directory");
+      } else if (!targetDir.isDirectory()) {
+        throw new SQLParserException("not a directory");
+      }
+      operator.setTargetDir(targetDir);
+    }

Review Comment:
   Added.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r947560488


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }

Review Comment:
   Why not check parent directory existence in another constructor? It's better to move check into MigrationLogWriter(File logFile) and let MigrationLogWriter(String logFileName) call this(new File(logFileName)).



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationLogWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationLogWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;

Review Comment:
   Shutdown these threads when the system exits.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationLogWriter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationLog writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationLogWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationLogWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  public MigrationLogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }
+    } catch (IOException e) {
+      logger.error("unable to write to migrate log");
+    }
+  }

Review Comment:
   Please flush after put log, logs may lost because they are only in the os buffer.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java:
##########
@@ -1555,6 +1556,72 @@ public void timedFlushSeqMemTable() {
     }
   }
 
+  /** iterate over TsFiles and migrate to targetDir if out of ttl */
+  public void checkMigration(MigrationTask task) {

Review Comment:
   Is it possible to move checkMigration and checkMigrateFile to MigrationTask?



##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -1254,6 +1257,46 @@ protected void getSeriesSchemas(InsertPlan insertPlan, VirtualStorageGroupProces
     }
   }
 
+  // push the migration info to migrationManager
+  public void setMigration(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    migrationManager.setMigrate(storageGroup, targetDir, ttl, startTime);
+    logger.info("start check migration task successfully.");
+  }
+
+  public void unsetMigration(long taskId, PartialPath storageGroup) {
+    if (taskId != -1) {

Review Comment:
   Are negative values less than -1 legal?Maybe taskId >= 0 is better.



##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;
+

Review Comment:
   Please add parameter migration_thread_num in the iotdb-datanode.properties and load it in the IoTDBDescriptor.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java:
##########
@@ -535,6 +535,33 @@ public boolean removeResourceFile() {
     return true;
   }
 
+  /**
+   * Move its data file, resource file, and modification file physically.
+   *
+   * @return migrated data file
+   */
+  public File migrate(File targetDir) {
+    // get the resource and mod files
+    File resourceFile = fsFactory.getFile(file.getPath() + RESOURCE_SUFFIX);
+    File modFile = fsFactory.getFile(file.getPath() + ModificationFile.FILE_SUFFIX);
+
+    // get the target file locations
+    File migratedFile = fsFactory.getFile(targetDir, file.getName());
+    File migratedResourceFile = fsFactory.getFile(targetDir, file.getName() + RESOURCE_SUFFIX);
+    File migratedModificationFile =
+        fsFactory.getFile(targetDir, file.getName() + ModificationFile.FILE_SUFFIX);
+
+    // move
+    fsFactory.moveFile(file, migratedFile);
+    if (resourceFile.exists()) {
+      fsFactory.moveFile(resourceFile, migratedResourceFile);
+    }
+    if (modFile.exists()) {
+      fsFactory.moveFile(modFile, migratedModificationFile);
+    }
+    return migratedFile;
+  }

Review Comment:
   This method isn't atomic. If the system crashes or exception occurs when this method is running, some files may still in the origin data directory, not in the target directory. How to recover our system from this scenario? Please think of a solution to this problem.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTask.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+
+import java.io.File;
+
+/** Data class for each Migration Task */
+public class MigrationTask {
+  private long taskId;
+  private PartialPath storageGroup;
+  private File targetDir;
+  private long startTime;
+  private long ttl;
+  private MigrationTaskStatus status = MigrationTaskStatus.READY;

Review Comment:
   Please add volatile keyword.



##########
server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java:
##########
@@ -408,6 +420,12 @@ public boolean processNonQuery(PhysicalPlan plan)
         return true;
       case SHOW_QUERY_RESOURCE:
         return processShowQueryResource();
+      case MIGRATION:
+        operateMigration((SetMigrationPlan) plan);

Review Comment:
   Please rename MIGRATION to SET_MIGRATION and operateMigration to operateSetMigration.



##########
server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java:
##########
@@ -760,6 +778,8 @@ protected QueryDataSet processShowQuery(ShowPlan showPlan, QueryContext context)
         return processShowPathsSetSchemaTemplate((ShowPathsSetTemplatePlan) showPlan);
       case PATHS_USING_SCHEMA_TEMPLATE:
         return processShowPathsUsingSchemaTemplate((ShowPathsUsingTemplatePlan) showPlan);
+      case MIGRATION:

Review Comment:
   Please rename MIGRATION to SHOW_MIGRATION.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r955569131


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();

Review Comment:
   Close these FileOutputStreams when system exits.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  @Override
+  public void close() {
+    for (FileOutputStream logFileStream : logOutputMap.values()) {
+      if (logFileStream != null) {
+        try {
+          logFileStream.close();
+        } catch (IOException e) {
+          logger.error("log file could not be closed");
+        }
+      }
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /**
+   * started the migration task, write to
+   *
+   * @return true if write log successful, false otherwise
+   */
+  public boolean startTask(long taskId, File targetDir) throws IOException {
+    FileOutputStream logFileOutput;
+    if (logOutputMap.containsKey(taskId) && logOutputMap.get(taskId) != null) {
+      logOutputMap.get(taskId).close();
+      logOutputMap.remove(taskId);
+    }
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+    if (logFile.exists()) {
+      // want an empty log file
+      logFile.delete();
+    }
+    if (!logFile.createNewFile()) {
+      // log file doesn't exist but cannot be created
+      return false;
+    }
+
+    logFileOutput = new FileOutputStream(logFile);
+    logOutputMap.put(taskId, logFileOutput);
+
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+
+    return true;
+  }
+
+  /**
+   * started migrating tsfile and its resource/mod files
+   *
+   * @return true if write log successful, false otherwise
+   */
+  public boolean start(long taskId, File tsfile) throws IOException {
+    FileOutputStream logFileOutput;
+    if (logOutputMap.containsKey(taskId) && logOutputMap.get(taskId) != null) {
+      logFileOutput = logOutputMap.get(taskId);
+    } else {
+      File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+      if (!logFile.exists()) {
+        if (!logFile.createNewFile()) {
+          // log file doesn't exist but cannot be created
+          return false;
+        }
+      }
+
+      logFileOutput = new FileOutputStream(logFile);
+      logOutputMap.put(taskId, logFileOutput);
+    }
+
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(long taskId) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, taskId + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }
+    if (logOutputMap.containsKey(taskId)) {
+      try {
+        logOutputMap.get(taskId).close();
+      } catch (IOException e) {
+        logger.error("could not close fileoutputstream for task {}", taskId);
+      }
+      logOutputMap.remove(taskId);
+    }
+  }
+
+  /** finish the unfinished MigrationTasks using log files under MIGRATING_LOG_DIR */
+  public void recover() {
+    for (File logFile : MIGRATING_LOG_DIR.listFiles()) {
+      try {
+        FileInputStream logFileInput = new FileInputStream(logFile);
+        String targetDirPath = ReadWriteIOUtils.readString(logFileInput);
+
+        File targetDir = SystemFileFactory.INSTANCE.getFile(targetDirPath);
+        while (logFileInput.available() > 0) {

Review Comment:
   1. FileInputStream.available() method is time-consuming, please avoid using this method.
   2. Delete log files after recovery.



##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;

Review Comment:
   Please keep consistent with iotdb-engine.properties



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {

Review Comment:
   It's better to rename MigratingFileLogManager, MigrationLogReader and MigrationLogWriter to make them read more understandable.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }

Review Comment:
   Excellent work 👍 



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r955575612


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }

Review Comment:
   Excellent work 👍 



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r957295147


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -865,6 +865,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 2;

Review Comment:
   Rename it to migrationThreadNum.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }

Review Comment:
   If you only need migration tasks, please don‘t return the map, you can return the copy of values.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



##########
server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java:
##########
@@ -785,6 +787,126 @@ public Operator visitUnsetSchemaTemplate(IoTDBSqlParser.UnsetSchemaTemplateConte
     return operator;
   }
 
+  // Set Migration
+  @Override
+  public Operator visitSetMigration(IoTDBSqlParser.SetMigrationContext ctx) {
+    SetMigrationOperator operator = new SetMigrationOperator(SQLConstant.TOK_SET);
+
+    if (ctx.storageGroup != null) {
+      operator.setStorageGroup(parsePrefixPath(ctx.storageGroup));
+    }
+    if (ctx.ttl != null) {
+      operator.setTTL(Long.parseLong(ctx.ttl.getText()));
+    }
+    if (ctx.startTime != null) {
+      operator.setStartTime(parseDateFormat(ctx.startTime.getText()));
+    }
+    if (ctx.targetDir != null) {
+      FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+      File targetDir = fsFactory.getFile(parseStringLiteral(ctx.targetDir.getText()));
+      if (!targetDir.exists()) {
+        throw new SQLParserException("unknown directory");
+      } else if (!targetDir.isDirectory()) {
+        throw new SQLParserException("not a directory");
+      }
+      operator.setTargetDir(targetDir);
+    }

Review Comment:
   1. Storage group, ttl, start time and target dir cannot be null.
   2. Add test to test null value.



##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java:
##########
@@ -512,6 +512,15 @@ public void loadProperties(Properties properties) throws UnknownHostException {
       conf.setConcurrentSubRawQueryThread(Runtime.getRuntime().availableProcessors());
     }
 
+    conf.setMigrationThread(
+        Integer.parseInt(
+            properties.getProperty(
+                "migration_thread_num", Integer.toString(conf.getMigrationThread()))));
+
+    if (conf.getMigrationThread() <= 0) {
+      conf.setMigrationThread(Runtime.getRuntime().availableProcessors());
+    }

Review Comment:
   Use default configuration 2



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");

Review Comment:
   Move "Migration-Check" and "Migration-Task" into ThreadName.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }

Review Comment:
   When you shutdown this manager, please release all resources.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTaskWriter.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationTasKWriter writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationTaskWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationTaskWriter(String logFileName) throws FileNotFoundException {
+    this(SystemFileFactory.INSTANCE.getFile(logFileName));
+  }
+
+  public MigrationTaskWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }

Review Comment:
   You can encapsulate these code into MigrationLog.serialize, like PhysicalPlan.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }

Review Comment:
   You can encapsulates this recover logic.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;

Review Comment:
   Why change current task id for an error migration task?



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationTaskWriter.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** MigrationTasKWriter writes the binary logs of MigrationTask into file using FileOutputStream */
+public class MigrationTaskWriter implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
+  private final File logFile;
+  private FileOutputStream logFileOutStream;
+
+  public MigrationTaskWriter(String logFileName) throws FileNotFoundException {
+    this(SystemFileFactory.INSTANCE.getFile(logFileName));
+  }
+
+  public MigrationTaskWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    if (!logFile.exists()) {
+      if (logFile.getParentFile() != null) {
+        if (logFile.getParentFile().mkdirs()) {
+          logger.info("created migrate log folder");
+        } else {
+          logger.info("create migrate log folder failed");
+        }
+      }
+    }
+    logFileOutStream = new FileOutputStream(logFile, true);
+  }
+
+  private void putLog(MigrationLog log) {
+    try {
+      int type = log.type.ordinal();
+      ReadWriteIOUtils.write((byte) type, logFileOutStream);
+      ReadWriteIOUtils.write(log.taskId, logFileOutStream);
+
+      if (log.type == MigrationLog.LogType.SET) {
+        ReadWriteIOUtils.write(log.storageGroup.getFullPath(), logFileOutStream);
+        ReadWriteIOUtils.write(log.targetDirPath, logFileOutStream);
+        ReadWriteIOUtils.write(log.startTime, logFileOutStream);
+        ReadWriteIOUtils.write(log.ttl, logFileOutStream);
+      }
+
+      logFileOutStream.flush();
+    } catch (IOException e) {
+      logger.error("unable to write to migrate log");
+    }
+  }
+
+  public void setMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log =
+        new MigrationLog(
+            MigrationLog.LogType.SET,
+            migrationTask.getTaskId(),
+            migrationTask.getStorageGroup(),
+            migrationTask.getTargetDir().getPath(),
+            migrationTask.getStartTime(),
+            migrationTask.getTTL());
+    putLog(log);
+  }
+
+  public void startMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.START, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void finishMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.FINISHED, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void unsetMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.CANCEL, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void pauseMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.PAUSE, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void unpauseMigration(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.RESUME, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  public void error(MigrationTask migrationTask) throws IOException {
+    MigrationLog log = new MigrationLog(MigrationLog.LogType.ERROR, migrationTask.getTaskId());
+    putLog(log);
+  }
+
+  @Override
+  public void close() throws Exception {
+    logFileOutStream.close();
+  }
+
+  public static class MigrationLog {
+    public LogType type;
+    public long taskId;
+    public PartialPath storageGroup;
+    public String targetDirPath;
+    public long startTime;
+    public long ttl;
+
+    public MigrationLog() {}
+
+    public MigrationLog(LogType type, long taskId) {
+      this.type = type;
+      this.taskId = taskId;
+    }
+
+    public MigrationLog(
+        LogType type,
+        long taskId,
+        PartialPath storageGroup,
+        String targetDirPath,
+        long startTime,
+        long ttl) {
+      this.type = type;
+      this.taskId = taskId;
+      this.storageGroup = storageGroup;
+      this.targetDirPath = targetDirPath;
+      this.startTime = startTime;
+      this.ttl = ttl;
+    }
+
+    public MigrationLog(LogType type, MigrationTask task) {
+      this.type = type;
+      this.taskId = task.getTaskId();
+      this.storageGroup = task.getStorageGroup();
+      this.targetDirPath = task.getTargetDir().getPath();
+      this.startTime = task.getStartTime();
+      this.ttl = task.getTTL();
+    }
+
+    public enum LogType {
+      SET,
+      CANCEL,
+      START,
+      PAUSE,
+      RESUME,
+      FINISHED,
+      ERROR
+    }

Review Comment:
   Maybe you can rename this enum to MigrationOperationType and make it an individual enum. So, you can use this enum to distinguish different operations.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as pauseMigrate using taskId except does not write to log */
+  public boolean pauseMigrationFromLog(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      migrationTasks.get(pauseTaskId).setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Unpause migration task from migrationTasks list using taskId
+   *
+   * @param unpauseTaskId taskId of task to unpause
+   * @return true if task with index exists and paused
+   */
+  public boolean resumeMigration(long unpauseTaskId) {
+    if (migrationTasks.containsKey(unpauseTaskId)) {
+      MigrationTask task = migrationTasks.get(unpauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to READY
+        task.setStatus(MigrationTask.MigrationTaskStatus.READY);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Resume migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId and paused.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean resumeMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED)) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to unset
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Cancel migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists, remove the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to remove
+   * @return true if exists task with storageGroup
+   */
+  public boolean cancelMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to UNSET
+        task.setStatus(MigrationTask.MigrationTaskStatus.UNSET);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as cancelMigrate using taskId except does not write to log */
+  public boolean cancelMigrationFromLog(long taskId) {
+    if (migrationTasks.containsKey(taskId)) {
+      migrationTasks.remove(taskId);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * pause migration task from migrationTasks list using taskId
+   *
+   * @param pauseTaskId taskId of task to pause
+   * @return true if task with taskId exists and pauseable
+   */
+  public boolean pauseMigration(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      MigrationTask task = migrationTasks.get(pauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+          || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Pause migration task from migrationTasks list using storage group. If multiple tasks with such
+   * storage group exists (and pauseable), pause the one with the lowest taskId.
+   *
+   * @param storageGroup sg for task to pause
+   * @return true if exists task with storageGroup and pauseable
+   */
+  public boolean pauseMigration(PartialPath storageGroup) {
+    for (MigrationTask task : migrationTasks.values()) {
+      if (task.getStorageGroup().getFullPath().equals(storageGroup.getFullPath())
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.pauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }
+        // change status to PAUSED
+        task.setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** same as pauseMigrate using taskId except does not write to log */
+  public boolean pauseMigrationFromLog(long pauseTaskId) {
+    if (migrationTasks.containsKey(pauseTaskId)) {
+      migrationTasks.get(pauseTaskId).setStatus(MigrationTask.MigrationTaskStatus.PAUSED);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Unpause migration task from migrationTasks list using taskId
+   *
+   * @param unpauseTaskId taskId of task to unpause
+   * @return true if task with index exists and paused
+   */
+  public boolean resumeMigration(long unpauseTaskId) {
+    if (migrationTasks.containsKey(unpauseTaskId)) {
+      MigrationTask task = migrationTasks.get(unpauseTaskId);
+      if (task.getStatus() == MigrationTask.MigrationTaskStatus.PAUSED) {
+        // write to log
+        try {
+          logWriter.unpauseMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;

Review Comment:
   1. Your code isn't thread-safe, the changes of your map, task id and log file shoud be atomic. Please use lock to protect them. If you use lock here, you can use HashMap instead of ConcurrentHashMap.
   2. Add test to concurrently set and unset migration tasks.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }
+
+  /** creates a copy of migrationTask and returns */
+  public ConcurrentHashMap<Long, MigrationTask> getMigrateTasks() {
+    return new ConcurrentHashMap<>(migrationTasks);
+  }
+
+  /** add migration task to migrationTasks */
+  public void setMigrate(PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    MigrationTask newTask =
+        new MigrationTask(currentTaskId, storageGroup, targetDir, ttl, startTime);
+    try {
+      logWriter.setMigration(newTask);
+    } catch (IOException e) {
+      logger.error("write log error");
+      return;
+    }
+    migrationTasks.put(currentTaskId, newTask);
+    currentTaskId++;
+  }
+
+  /** add migration task to migrationTasks from log, does not write to log */
+  public void setMigrationFromLog(
+      long taskId, PartialPath storageGroup, File targetDir, long ttl, long startTime) {
+    if (currentTaskId > taskId) {
+      logger.error("set migration error, current index larger than log index");
+    }
+
+    MigrationTask newTask = new MigrationTask(taskId, storageGroup, targetDir, ttl, startTime);
+    migrationTasks.put(taskId, newTask);
+    currentTaskId = taskId + 1;
+  }
+
+  /**
+   * unset migration task from migrationTasks list using taskId
+   *
+   * @param unsetTaskId taskId of task to remove
+   * @return true if task with taskId exists, false otherwise
+   */
+  public boolean cancelMigration(long unsetTaskId) {
+    if (migrationTasks.containsKey(unsetTaskId)) {
+      MigrationTask task = migrationTasks.get(unsetTaskId);
+      if (task.getTaskId() == unsetTaskId
+          && (task.getStatus() == MigrationTask.MigrationTaskStatus.READY
+              || task.getStatus() == MigrationTask.MigrationTaskStatus.RUNNING)) {
+        // write to log
+        try {
+          logWriter.unsetMigration(task);
+        } catch (IOException e) {
+          logger.error("write log error");
+        }

Review Comment:
   Please return here.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962396447


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;

Review Comment:
   1. Fixed.
   2. Added.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950862672


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationLogWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationLogWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;

Review Comment:
   Fixed



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationLogWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationLogWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r957338549


##########
server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java:
##########
@@ -785,6 +787,126 @@ public Operator visitUnsetSchemaTemplate(IoTDBSqlParser.UnsetSchemaTemplateConte
     return operator;
   }
 
+  // Set Migration
+  @Override
+  public Operator visitSetMigration(IoTDBSqlParser.SetMigrationContext ctx) {
+    SetMigrationOperator operator = new SetMigrationOperator(SQLConstant.TOK_SET);
+
+    if (ctx.storageGroup != null) {
+      operator.setStorageGroup(parsePrefixPath(ctx.storageGroup));
+    }
+    if (ctx.ttl != null) {
+      operator.setTTL(Long.parseLong(ctx.ttl.getText()));
+    }
+    if (ctx.startTime != null) {
+      operator.setStartTime(parseDateFormat(ctx.startTime.getText()));
+    }
+    if (ctx.targetDir != null) {
+      FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+      File targetDir = fsFactory.getFile(parseStringLiteral(ctx.targetDir.getText()));
+      if (!targetDir.exists()) {
+        throw new SQLParserException("unknown directory");
+      } else if (!targetDir.isDirectory()) {
+        throw new SQLParserException("not a directory");
+      }
+      operator.setTargetDir(targetDir);
+    }

Review Comment:
   Add test to test null value.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r954417591


##########
server/src/assembly/resources/conf/iotdb-engine.properties:
##########
@@ -275,6 +275,10 @@ timestamp_precision=ms
 # Datatype: int
 # concurrent_query_thread=16
 
+# How many threads can concurrently run migration tasks. When <= 0, use CPU core number.
+# Datatype: int
+# migration_thread_num=16

Review Comment:
   Changed to 2.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962395892


##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -587,6 +591,8 @@ public void shutdown(long milliseconds) throws ShutdownException {
     shutdownTimedService(tsFileTimedCloseCheckThread, "TsFileTimedCloseCheckThread");
     recoveryThreadPool.shutdownNow();
     processorMap.clear();
+    migrationManager.shutdown();
+    TsFileMigrationLogger.getInstance().close();

Review Comment:
   Moved.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, TsFileMigrationLogger writes files
+ * to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) , then
+ * deletes it after the task has finished operation.
+ */
+public class TsFileMigrationLogger implements AutoCloseable {

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Support Data Archiving

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r976656038


##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration
+setMigration
+    : SET MIGRATION TO storageGroup=prefixPath startTime=DATETIME_LITERAL ttl=INTEGER_LITERAL targetDir=STRING_LITERAL
+    | SET MIGRATION TO setMigrationClause*
+    ;
+
+setMigrationClause

Review Comment:
   Changed.



##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration

Review Comment:
   Changed.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r947417173


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;
+

Review Comment:
   Please add parameter migration_thread_num in the iotdb-engine.properties and load it in the IoTDBDescriptor.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950857525


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -851,6 +851,9 @@ public class IoTDBConfig {
   // The max record num returned in one schema query.
   private int schemaQueryFetchSize = 10000000;
 
+  /** number of threads given to migration tasks */
+  private int migrationThread = 1;
+

Review Comment:
   fixed



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950862604


##########
server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java:
##########
@@ -408,6 +420,12 @@ public boolean processNonQuery(PhysicalPlan plan)
         return true;
       case SHOW_QUERY_RESOURCE:
         return processShowQueryResource();
+      case MIGRATION:
+        operateMigration((SetMigrationPlan) plan);

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java:
##########
@@ -760,6 +778,8 @@ protected QueryDataSet processShowQuery(ShowPlan showPlan, QueryContext context)
         return processShowPathsSetSchemaTemplate((ShowPathsSetTemplatePlan) showPlan);
       case PATHS_USING_SCHEMA_TEMPLATE:
         return processShowPathsUsingSchemaTemplate((ShowPathsUsingTemplatePlan) showPlan);
+      case MIGRATION:

Review Comment:
   Fixed



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r956523921


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) ,
+ * then deletes it after the task has finished operation.
+ */
+public class MigratingFileLogManager implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962396160


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/TsFileMigrationLogger.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, TsFileMigrationLogger writes files
+ * to migratingFileDir when a migration task migrates a tsFile (and its resource/mod files) , then
+ * deletes it after the task has finished operation.
+ */
+public class TsFileMigrationLogger implements AutoCloseable {
+
+  // taskId -> MigratingFileLog
+  ConcurrentHashMap<Long, FileOutputStream> logOutputMap = new ConcurrentHashMap<>();

Review Comment:
   1. Fixed.
   2. Moved TsFileMigrationLogger responsibilities into MigrationTask.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r962396355


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,478 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.migration.MigrationTaskWriter.MigrationLog;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private MigrationTaskWriter logWriter;
+
+  // taskId -> MigrationTask
+  private ConcurrentHashMap<Long, MigrationTask> migrationTasks = new ConcurrentHashMap<>();
+  // the current largest MigrationTask id + 1, used to create new tasks
+  private long currentTaskId = 0;
+  // single thread to iterate through migrationTasks and check start
+  private ScheduledExecutorService migrationCheckThread;
+  // multiple threads to run the tasks
+  private ExecutorService migrationTaskThreadPool;
+
+  private boolean initialized = false;
+  private static final long MIGRATE_CHECK_INTERVAL = 60 * 1000L;
+  private static final String LOG_FILE_NAME =
+      Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "log.bin")
+          .toString();
+
+  protected MigrationManager() {
+    init();
+  }
+
+  // singleton
+  private static class MigrationManagerHolder {
+    private MigrationManagerHolder() {}
+
+    private static final MigrationManager INSTANCE = new MigrationManager();
+  }
+
+  public static MigrationManager getInstance() {
+    return MigrationManagerHolder.INSTANCE;
+  }
+
+  public synchronized void init() {
+    if (initialized) {
+      return;
+    }
+
+    try {
+      logWriter = new MigrationTaskWriter(LOG_FILE_NAME);
+    } catch (FileNotFoundException e) {
+      logger.error("Cannot find/create log for migration.");
+    }
+
+    // read from logReader
+    try {
+      MigrationTaskReader logReader = new MigrationTaskReader(LOG_FILE_NAME);
+      Set<Long> errorSet = new HashSet<>();
+
+      while (logReader.hasNext()) {
+        MigrationLog log = logReader.next();
+
+        switch (log.type) {
+          case SET:
+            setMigrationFromLog(
+                log.taskId,
+                log.storageGroup,
+                FSFactoryProducer.getFSFactory().getFile(log.targetDirPath),
+                log.ttl,
+                log.startTime);
+            break;
+          case CANCEL:
+            cancelMigrationFromLog(log.taskId);
+            break;
+          case START:
+            // if task started but didn't finish, then error occurred
+            errorSet.add(log.taskId);
+            break;
+          case PAUSE:
+            errorSet.remove(log.taskId);
+            pauseMigrationFromLog(log.taskId);
+            break;
+          case RESUME:
+            resumeMigrationFromLog(log.taskId);
+            break;
+          case FINISHED:
+            // finished task => remove from list and remove from potential error task
+            errorSet.remove(log.taskId);
+            migrationTasks.remove(log.taskId);
+            finishFromLog(log.taskId);
+            break;
+          case ERROR:
+            // already put error in log
+            errorSet.remove(log.taskId);
+            errorFromLog(log.taskId);
+            break;
+          default:
+            logger.error("read migration log: unknown type");
+        }
+      }
+
+      // for each task in errorSet, the task started but didn't finish (an error)
+      for (long errIndex : errorSet) {
+        if (migrationTasks.containsKey(errIndex)) {
+          // write to log and set task in ERROR in memory
+          logWriter.error(migrationTasks.get(errIndex));
+          errorFromLog(errIndex);
+        } else {
+          logger.error("unknown error index");
+        }
+      }
+
+    } catch (IOException e) {
+      logger.error("Cannot read log for migration.");
+    }
+
+    migrationCheckThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("Migration-Check");
+    migrationCheckThread.scheduleAtFixedRate(
+        this::checkMigration,
+        MIGRATE_CHECK_INTERVAL,
+        MIGRATE_CHECK_INTERVAL,
+        TimeUnit.MILLISECONDS);
+
+    migrationTaskThreadPool =
+        IoTDBThreadPoolFactory.newFixedThreadPool(config.getMigrationThread(), "Migration-Task");
+    logger.info("start migration check thread successfully.");
+    initialized = true;
+  }
+
+  /** close the log writer */
+  public void clear() {
+    try {
+      logWriter.close();
+    } catch (Exception e) {
+      logger.error("Cannot close migration log writer, because:", e);
+    }
+  }
+
+  /** shutdown all threads used */
+  public void shutdown() {
+    migrationCheckThread.shutdown();
+    migrationTaskThreadPool.shutdown();
+  }

Review Comment:
   Fixed.



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

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

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


[GitHub] [iotdb] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r950860996


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java:
##########
@@ -535,6 +535,33 @@ public boolean removeResourceFile() {
     return true;
   }
 
+  /**
+   * Move its data file, resource file, and modification file physically.
+   *
+   * @return migrated data file
+   */
+  public File migrate(File targetDir) {
+    // get the resource and mod files
+    File resourceFile = fsFactory.getFile(file.getPath() + RESOURCE_SUFFIX);
+    File modFile = fsFactory.getFile(file.getPath() + ModificationFile.FILE_SUFFIX);
+
+    // get the target file locations
+    File migratedFile = fsFactory.getFile(targetDir, file.getName());
+    File migratedResourceFile = fsFactory.getFile(targetDir, file.getName() + RESOURCE_SUFFIX);
+    File migratedModificationFile =
+        fsFactory.getFile(targetDir, file.getName() + ModificationFile.FILE_SUFFIX);
+
+    // move
+    fsFactory.moveFile(file, migratedFile);
+    if (resourceFile.exists()) {
+      fsFactory.moveFile(resourceFile, migratedResourceFile);
+    }
+    if (modFile.exists()) {
+      fsFactory.moveFile(modFile, migratedModificationFile);
+    }
+    return migratedFile;
+  }

Review Comment:
   Fixed by adding MigratingFileLogManager that keeps track of file migrations, and provides the ability to recover.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r955529416


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigratingFileLogManager.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import static org.apache.iotdb.db.metadata.idtable.IDTable.config;
+import static org.apache.iotdb.db.metadata.idtable.IDTable.logger;
+
+/**
+ * To assure that migration of tsFiles is pesudo-atomic operator, MigratingFileLogManager writes
+ * files to migratingFileDir when a tsFile (and its resource/mod files) are being migrated, then
+ * deletes it after it has finished operation.
+ */
+public class MigratingFileLogManager {
+
+  private static File MIGRATING_LOG_DIR =
+      SystemFileFactory.INSTANCE.getFile(
+          Paths.get(FilePathUtils.regularizePath(config.getSystemDir()), "migration", "migrating")
+              .toString());
+
+  protected MigratingFileLogManager() {
+    if (MIGRATING_LOG_DIR == null) {
+      logger.error("MIGRATING_LOG_DIR is null");
+    }
+
+    if (!MIGRATING_LOG_DIR.exists()) {
+      if (MIGRATING_LOG_DIR.mkdirs()) {
+        logger.info("MIGRATING_LOG_DIR {} created successfully", MIGRATING_LOG_DIR);
+      } else {
+        logger.error("MIGRATING_LOG_DIR {} create error", MIGRATING_LOG_DIR);
+      }
+      return;
+    }
+
+    if (!MIGRATING_LOG_DIR.isDirectory()) {
+      logger.error("{} already exists but is not directory", MIGRATING_LOG_DIR);
+    }
+  }
+
+  // singleton
+  private static class MigratingFileLogManagerHolder {
+    private MigratingFileLogManagerHolder() {}
+
+    private static final MigratingFileLogManager INSTANCE = new MigratingFileLogManager();
+  }
+
+  public static MigratingFileLogManager getInstance() {
+    return MigratingFileLogManagerHolder.INSTANCE;
+  }
+
+  /** started migrating tsfile and its resource/mod files */
+  public boolean start(File tsfile, File targetDir) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (!logFile.createNewFile()) {
+      return false;
+    }
+
+    FileOutputStream logFileOutput = new FileOutputStream(logFile);
+    ReadWriteIOUtils.write(tsfile.getAbsolutePath(), logFileOutput);
+    ReadWriteIOUtils.write(targetDir.getAbsolutePath(), logFileOutput);
+    logFileOutput.flush();
+    logFileOutput.close();
+
+    return true;
+  }
+
+  /** finished migrating tsfile and related files */
+  public void finish(File tsfile) {
+    File logFile = SystemFileFactory.INSTANCE.getFile(MIGRATING_LOG_DIR, tsfile.getName() + ".log");
+    if (logFile.exists()) {
+      logFile.delete();
+    }

Review Comment:
   I'm not sure how to sync the log for implementation 1 when there are concurrent tasks, so instead I tested:
   
   1. Without Logs (baseline).
   2. Original Implementation.
   3. One log for each migration task, append to log for each tsfile migration.
   4. One log for each migration task, overwrite log for each tsfile migration.
   
   Tested on 10000 tsfiles.
   
   | Implementation | Migration Time (ms) | Recovery Time (ms) |
   | ----------- | ------- | ----------- |
   | Without Logs | 400 | - |
   | Original | 1400 | 2 |
   | Append Log | 450 | 900 |
   | Overwrite Log | 800 | 2 |
   
   Since migrating is more common and recovery is sparse, changed to method 3 (one log per task, append 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.

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

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


[GitHub] [iotdb] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r964324467


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private final ReentrantLock lock = new ReentrantLock();
+  // region lock resources
+  private MigrationOperateWriter logWriter;
+  // taskId -> MigrationTask
+  List<MigrationTask> migrationTasks = new ArrayList<>();

Review Comment:
   If you want to sort elements, maybe you can use PriorityQueue here.



-- 
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] HeimingZ commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
HeimingZ commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r964324467


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private final ReentrantLock lock = new ReentrantLock();
+  // region lock resources
+  private MigrationOperateWriter logWriter;
+  // taskId -> MigrationTask
+  List<MigrationTask> migrationTasks = new ArrayList<>();

Review Comment:
   If you want to sort elements, maybe you can use TreeSet here.



-- 
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] sogood99 commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Separation

Posted by GitBox <gi...@apache.org>.
sogood99 commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r970427474


##########
server/src/main/java/org/apache/iotdb/db/engine/migration/MigrationManager.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.engine.migration;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * MigrationManager keep tracks of all Migration Tasks, creates the threads to check/run the
+ * MigrationTask.
+ */
+public class MigrationManager {
+  private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private final ReentrantLock lock = new ReentrantLock();
+  // region lock resources
+  private MigrationOperateWriter logWriter;
+  // taskId -> MigrationTask
+  List<MigrationTask> migrationTasks = new ArrayList<>();

Review Comment:
   Changed.



-- 
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] qiaojialin commented on a diff in pull request #6989: [To rel/0.13][IOTDB-3882] Cold Data Archiving

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on code in PR #6989:
URL: https://github.com/apache/iotdb/pull/6989#discussion_r975990337


##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration
+setMigration
+    : SET MIGRATION TO storageGroup=prefixPath startTime=DATETIME_LITERAL ttl=INTEGER_LITERAL targetDir=STRING_LITERAL
+    | SET MIGRATION TO setMigrationClause*
+    ;
+
+setMigrationClause

Review Comment:
   ```suggestion
   setArchiveClause
   ```



##########
integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMigrationIT.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.integration;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.utils.FileUtils;
+import org.apache.iotdb.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBMigrationIT {

Review Comment:
   ```suggestion
   public class IoTDBArchiveIT {
   ```



##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration
+setMigration
+    : SET MIGRATION TO storageGroup=prefixPath startTime=DATETIME_LITERAL ttl=INTEGER_LITERAL targetDir=STRING_LITERAL
+    | SET MIGRATION TO setMigrationClause*
+    ;
+
+setMigrationClause
+    : STORAGE_GROUP     OPERATOR_EQ storageGroup=prefixPath
+    | START_TIME        OPERATOR_EQ startTime=DATETIME_LITERAL
+    | TTL               OPERATOR_EQ ttl=INTEGER_LITERAL
+    | TARGET_DIR        OPERATOR_EQ targetDir=STRING_LITERAL
+    ;
+
+// Cancel Migration
+cancelMigration
+    : CANCEL MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Pause Migration
+pauseMigration
+    : PAUSE MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Unpause/Resume migration
+resumeMigration
+    : RESUME MIGRATION (ON storageGroup=prefixPath | taskId=INTEGER_LITERAL)
+    ;
+
+// Show Migration
+showMigration
+    : SHOW MIGRATION ON prefixPath (COMMA prefixPath)*

Review Comment:
   ```suggestion
       : SHOW ARCHIVE ON prefixPath (COMMA prefixPath)*
   ```



##########
server/src/assembly/resources/conf/iotdb-engine.properties:
##########
@@ -275,6 +275,10 @@ timestamp_precision=ms
 # Datatype: int
 # concurrent_query_thread=16
 
+# How many threads can concurrently run migration tasks. When <= 0, use CPU core number.
+# Datatype: int
+# migration_thread_num=2

Review Comment:
   ```suggestion
   # archive_thread_num=2
   ```



##########
antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4:
##########
@@ -330,6 +332,44 @@ countNodes
     : COUNT NODES prefixPath LEVEL OPERATOR_EQ INTEGER_LITERAL
     ;
 
+// Set Migration

Review Comment:
   ```suggestion
   // Set Archive
   ```



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