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/03 08:59:07 UTC

[GitHub] [iotdb] lpf4254302 opened a new pull request, #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

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

   ## Description
   ````
   Target:
   Support "Modify Encoding Type and Compression Type" command
   Application scenarios:
   In IoTDB application projects, reasonable setting of encoding and compression algorithms can effectively reduce disk space occupancy and reduce server costs in disguise. Modifying encoding and compression algorithms is an ideal method.
   Example:
   The physical quantity root.sg1.device_1.m1, the initial encoding type is PLAIN, and the data characteristics are straight-up data 1,2,3,4,5.....
   After a long period of accumulation, the data occupies about 1G of hard disk space. Now after modifying the encoding type to TS_2DIFF, the data occupying the hard disk space can be reduced to less than 100M.
   ````
   ### plan selection
   #### About the selection of single/multiple physical quantity modification
   ````
   Option 1: A single command only modifies the encoding type and compression type of a single physical quantity
   Option 2: A single command supports batch modification of physical quantity encoding type and compression type
   Conclusion: Option 1 is selected for the first version, and Option 2 is supported after the command verification is passed.
   ````
   #### About the selection of the command affecting the data range
   ````
   Option 1: Affect the newly inserted data after modification
   Option 2: Affect all sealed, unsealed and newly inserted data after modification
   Conclusion: The first version chooses option 2. After the command is executed successfully, you can immediately see the disk changes
   ````
   ### Merge related code changes
   ````
   Before developing the function, it is necessary to modify the original merged code to adapt to this function
   1. When the same physical quantity has different encoding types or compression types in the tsfile file, the merged tsfile will be damaged, and this problem needs to be fixed
   2. Since the merge process is a mutually exclusive operation, it is necessary to increase the lock control
   ````
   ### Modify the encoding type and compression type positive process
   ````
   1. Verify the request parameters before execution, including some non-null verification, cluster status verification, etc.
   2. Modify the encoding type and compression type in the schema
   3. Void the schema cache
   4. Find the storage group
   5. Perform modification operations by virtual storage group
   6. Force close working TsFileProcessors
   7. Get rewriteLock
   8. Generate alter log
   9. Rewrite the ordered and unordered tsfile data operations separately
   9.1. (Un)sequenceListByTimePartition
   9.2. Traverse tsFileResource
   9.3. Filter unexecuted tsFileResource (recovery)
   9.4. Generate targetTsFileResource
   9.5. Rewrite tsFileResource
   9.5.1. Acquire tsFileResource read lock
   9.5.2, read device list
   9.5.3. startChunkGroup
   9.5.4, read chunks
   9.5.5. If the measurement is not modified by the target, write the chunk directly
   9.5.6. If it is a measurement modified by the target, read pages and points one by one, re-encode and compress them before writing
   9.5.7. endChunkGroup
   9.5.8. endFile
   9.5.9. Release the tsFileResource read lock
   9.6. Rename the file to .tsfile->.alter.old .alter->.tsfile
   9.7. Replace tsFileResource and targetTsFileResource
   9.8. Delete the original tsfile related files (.tsfile .resource .mods)
   10. Delete alter log
   11. Release rewriteLock
   ````
   ### Recovery operation after schema modification
   mlog adds AlterTimeSeriesPlan and implements recovery method at the same time
   
   ### Service restart recovery operation
   ````
   1. Determine whether RecoverAlter is required before RecoverCompaction
   2. Execute recoverAlter before initCompaction
   
   recoverAlter method flow
   1. Analyze alter.log to get a list of unfinished tsfiles
   2. Check the list of unfinished tsfiles and perform pre-repair operations
   3. Rewrite the tsfile operation
   
   Pre-Recovery Action Policy
   Incomplete tsfile status:
   1. There is no .tsfile
   1.1, .alter.old exists and .alter exists - wait for completion
   1.2, only .alter.old exists - system exception
   1.3, only exists .alter - system exception
   2. There is .tsfile
   2.1, exists. alter - writing
   2.2, exist.alter.old - wait for delete
   2.3, does exist - not started
   ````
   ### Need to continue to improve the content
   ````
   1. Aligned time series data rewrite optimization
   2. Implementation of RSchemaRegion, SchemaRegionSchemaFileImpl related methods
   3. Support for clusters
   4. The tsfile rewrite operation is changed to asynchronous execution
   5. Support batch modification of physical quantities
   ````
   
   
   <hr>
   
   This PR has:
   - [√ ] been self-reviewed.
       - [√ ] concurrent read
       - [√ ] concurrent write
       - [√ ] concurrent read and write
   - [√ ] added documentation for new or modified features or behaviors.
   - [√ ] added Javadocs for most classes and all non-trivial methods.
   - [√ ] added comments explaining the "why" and the intent of the code wherever would not be obvious
     for an unfamiliar reader.
   - [√ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold
     for code coverage.
   - [√ ] added integration tests.
   - [ ] been tested in a test IoTDB cluster.
   
   <hr>
   
   ##### Key changed/added classes (or packages if there are too many classes) in this PR
   PlanExecutor
   LocalSchemaProcessor
   SchemaRegionSchemaFileImpl
   DataRegion
   org.apache.iotdb.db.engine.alter
   IoTDBSqlParser.g4
   CrossSpaceCompactionTask.java
   SingleSeriesCompactionExecutor.java
   InnerSpaceCompactionTask.java
   TsFileManager
   MeasurementMNode


-- 
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] lpf4254302 commented on a diff in pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

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


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)

Review Comment:
   fullPath
       : ROOT (DOT nodeNameWithoutWildcard)*
       ;
   FullPath does not support wildcards. There is no general method for determining wildcards. Do you need to add a wildcard method?



-- 
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] lpf4254302 commented on a diff in pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

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


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {
+      if (!findUndoneResourcesAndRemove(tsFileResource, undoneFiles)) {
+        continue;
+      }
+      if (tsFileResource == null || !tsFileResource.isClosed()) {
+        return;
+      }
+      try {
+        logger.info(
+            "[alter timeseries] {} rewriteDataInTsFile:{}, fileSize:{} start",
+            logKey,
+            tsFileResource.getTsFilePath(),
+            tsFileResource.getTsFileSize());
+        // Generate the target tsFileResource
+        TsFileResource targetTsFileResource =
+            TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource);
+        // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file
+        TsFileRewriteExcutor tsFileRewriteExcutor =
+            new TsFileRewriteExcutor(
+                tsFileResource,
+                targetTsFileResource,
+                fullPath,
+                curEncoding,
+                curCompressionType,
+                timePartition,
+                sequence);
+        tsFileRewriteExcutor.execute();
+        // .tsfile->.alter.old .alter->.tsfile
+        logger.debug("[alter timeseries] {} move tsfile", logKey);
+        tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX);
+        targetTsFileResource.moveTsFile(IoTDBConstant.ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX);
+        // replace
+        logger.debug("[alter timeseries] {} replace tsfile", logKey);
+        if (sequence) {
+          tsFileManager.replace(
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),

Review Comment:
   resolve



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {
+      if (!findUndoneResourcesAndRemove(tsFileResource, undoneFiles)) {
+        continue;
+      }
+      if (tsFileResource == null || !tsFileResource.isClosed()) {
+        return;
+      }
+      try {
+        logger.info(
+            "[alter timeseries] {} rewriteDataInTsFile:{}, fileSize:{} start",
+            logKey,
+            tsFileResource.getTsFilePath(),
+            tsFileResource.getTsFileSize());
+        // Generate the target tsFileResource
+        TsFileResource targetTsFileResource =
+            TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource);
+        // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file
+        TsFileRewriteExcutor tsFileRewriteExcutor =
+            new TsFileRewriteExcutor(
+                tsFileResource,
+                targetTsFileResource,
+                fullPath,
+                curEncoding,
+                curCompressionType,
+                timePartition,
+                sequence);
+        tsFileRewriteExcutor.execute();
+        // .tsfile->.alter.old .alter->.tsfile
+        logger.debug("[alter timeseries] {} move tsfile", logKey);
+        tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX);
+        targetTsFileResource.moveTsFile(IoTDBConstant.ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX);
+        // replace
+        logger.debug("[alter timeseries] {} replace tsfile", logKey);
+        if (sequence) {
+          tsFileManager.replace(
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),
+              timePartition,
+              true);
+        } else {
+          tsFileManager.replace(
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),

Review Comment:
   resolve



-- 
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] choubenson commented on a diff in pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

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


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -3662,6 +4071,38 @@ public void setAllowCompaction(boolean allowCompaction) {
     this.tsFileManager.setAllowCompaction(allowCompaction);
   }
 
+  /** delete tsfile, copy from CompactionUtils TODO We need to rename CompactionUtils */
+  public static boolean deleteTsFile(TsFileResource seqFile, String storageGroupName) {
+    try {
+      logger.info("{} Start to delete TsFile {}", storageGroupName, seqFile.getTsFilePath());
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
+      //      seqFile.setStatus(TsFileResourceStatus.DELETED);
+      seqFile.delete();

Review Comment:
   How about using `seqFile.remove()` to delete TsFiles, mods file and resource file, instead of using another method to delete mods file?



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {
+      if (!findUndoneResourcesAndRemove(tsFileResource, undoneFiles)) {
+        continue;
+      }
+      if (tsFileResource == null || !tsFileResource.isClosed()) {
+        return;
+      }
+      try {
+        logger.info(
+            "[alter timeseries] {} rewriteDataInTsFile:{}, fileSize:{} start",
+            logKey,
+            tsFileResource.getTsFilePath(),
+            tsFileResource.getTsFileSize());
+        // Generate the target tsFileResource
+        TsFileResource targetTsFileResource =
+            TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource);
+        // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file
+        TsFileRewriteExcutor tsFileRewriteExcutor =
+            new TsFileRewriteExcutor(
+                tsFileResource,
+                targetTsFileResource,
+                fullPath,
+                curEncoding,
+                curCompressionType,
+                timePartition,
+                sequence);
+        tsFileRewriteExcutor.execute();
+        // .tsfile->.alter.old .alter->.tsfile
+        logger.debug("[alter timeseries] {} move tsfile", logKey);
+        tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX);
+        targetTsFileResource.moveTsFile(IoTDBConstant.ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX);
+        // replace
+        logger.debug("[alter timeseries] {} replace tsfile", logKey);
+        if (sequence) {
+          tsFileManager.replace(
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),
+              timePartition,
+              true);
+        } else {
+          tsFileManager.replace(
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),

Review Comment:
   remove `new ArrayList<>()`



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -3662,6 +4071,38 @@ public void setAllowCompaction(boolean allowCompaction) {
     this.tsFileManager.setAllowCompaction(allowCompaction);
   }
 
+  /** delete tsfile, copy from CompactionUtils TODO We need to rename CompactionUtils */
+  public static boolean deleteTsFile(TsFileResource seqFile, String storageGroupName) {
+    try {
+      logger.info("{} Start to delete TsFile {}", storageGroupName, seqFile.getTsFilePath());
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
+      //      seqFile.setStatus(TsFileResourceStatus.DELETED);
+      seqFile.delete();
+    } catch (IOException e) {
+      logger.error(e.getMessage(), e);
+      return false;
+    }
+    return true;
+  }
+
+  /** Delete all modification files for source files TODO We need to rename CompactionUtils */
+  public static void deleteModificationForSourceFile(
+      Collection<TsFileResource> sourceFiles, String storageGroupName) throws IOException {
+    logger.info("{} Start to delete modifications of source files", storageGroupName);
+    for (TsFileResource tsFileResource : sourceFiles) {
+      ModificationFile compactionModificationFile =
+          ModificationFile.getCompactionMods(tsFileResource);
+      if (compactionModificationFile.exists()) {
+        compactionModificationFile.remove();
+      }

Review Comment:
   It's necessary to delete compaction mods file, because compaction mods file is only produced during compaction.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");

Review Comment:
   Change to `throw new IOException("Alter failed. Please do not alter until the old files settled.");`



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");

Review Comment:
   Change to `throw new IOException("Alter failed. Please do not alter until the old files upgraded.");`



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {
+      if (!findUndoneResourcesAndRemove(tsFileResource, undoneFiles)) {
+        continue;
+      }
+      if (tsFileResource == null || !tsFileResource.isClosed()) {
+        return;
+      }
+      try {
+        logger.info(
+            "[alter timeseries] {} rewriteDataInTsFile:{}, fileSize:{} start",
+            logKey,
+            tsFileResource.getTsFilePath(),
+            tsFileResource.getTsFileSize());
+        // Generate the target tsFileResource
+        TsFileResource targetTsFileResource =
+            TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource);
+        // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file
+        TsFileRewriteExcutor tsFileRewriteExcutor =
+            new TsFileRewriteExcutor(
+                tsFileResource,
+                targetTsFileResource,
+                fullPath,
+                curEncoding,
+                curCompressionType,
+                timePartition,
+                sequence);
+        tsFileRewriteExcutor.execute();
+        // .tsfile->.alter.old .alter->.tsfile
+        logger.debug("[alter timeseries] {} move tsfile", logKey);
+        tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX);
+        targetTsFileResource.moveTsFile(IoTDBConstant.ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX);
+        // replace
+        logger.debug("[alter timeseries] {} replace tsfile", logKey);
+        if (sequence) {
+          tsFileManager.replace(
+              new ArrayList<>(Collections.singletonList(tsFileResource)),
+              Collections.emptyList(),
+              new ArrayList<>(Collections.singletonList(targetTsFileResource)),

Review Comment:
   remove `new ArrayList<>()`



##########
server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewriteExcutor.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.alter;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.utils.CommonUtils;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.read.TsFileAlignedSeriesReaderIterator;
+import org.apache.iotdb.tsfile.read.TsFileDeviceIterator;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.IChunkReader;
+import org.apache.iotdb.tsfile.read.reader.IPointReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** This class is used to rewrite one tsfile with current encoding & compressionType. */
+public class TsFileRewriteExcutor {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteExcutor.class);
+
+  private final TsFileResource tsFileResource;
+  private final TsFileResource targetTsFileResource;
+  private final PartialPath fullPath;
+  private final TSEncoding curEncoding;
+  private final CompressionType curCompressionType;
+  private long timePartition;
+  private boolean sequence;
+  // record the min time and max time to update the target resource
+  private long minStartTimestamp = Long.MAX_VALUE;
+  private long maxEndTimestamp = Long.MIN_VALUE;
+
+  public TsFileRewriteExcutor(
+      TsFileResource tsFileResource,
+      TsFileResource targetTsFileResource,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence) {
+    this.tsFileResource = tsFileResource;
+    this.targetTsFileResource = targetTsFileResource;
+    this.fullPath = fullPath;
+    this.curEncoding = curEncoding;
+    this.curCompressionType = curCompressionType;
+    this.timePartition = timePartition;
+    this.sequence = sequence;
+  }
+
+  /** This function execute the rewrite task */
+  public void execute() throws IOException {
+
+    tsFileResource.tryReadLock();
+    try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFilePath());
+        TsFileIOWriter writer = new TsFileIOWriter(targetTsFileResource.getTsFile())) {
+      // read devices
+      TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned();
+      while (deviceIterator.hasNext()) {
+        minStartTimestamp = Long.MAX_VALUE;
+        maxEndTimestamp = Long.MIN_VALUE;
+        Pair<String, Boolean> deviceInfo = deviceIterator.next();
+        String device = deviceInfo.left;
+        boolean aligned = deviceInfo.right;
+        // write chunkGroup header
+        writer.startChunkGroup(device);
+        boolean isTargetDevice = fullPath.getDevice().equals(device);
+        String targetMeasurement = fullPath.getMeasurement();
+        // write chunk & page data
+        if (aligned) {
+          rewriteAlgined(reader, writer, device, isTargetDevice, targetMeasurement);
+        } else {
+          rewriteNotAligned(device, reader, writer, targetMeasurement, isTargetDevice);
+        }
+        // chunkGroup end
+        writer.endChunkGroup();
+
+        targetTsFileResource.updateStartTime(device, minStartTimestamp);
+        targetTsFileResource.updateEndTime(device, maxEndTimestamp);
+      }
+
+      targetTsFileResource.updatePlanIndexes(tsFileResource);
+      // write index,bloom,footer, end file
+      writer.endFile();
+      targetTsFileResource.close();
+    } finally {
+      tsFileResource.readUnlock();
+    }
+  }
+
+  private void rewriteAlgined(
+      TsFileSequenceReader reader,
+      TsFileIOWriter writer,
+      String device,
+      boolean isTargetDevice,
+      String targetMeasurement)
+      throws IOException {
+    List<AlignedChunkMetadata> alignedChunkMetadatas = reader.getAlignedChunkMetadata(device);
+    if (alignedChunkMetadatas == null || alignedChunkMetadatas.isEmpty()) {
+      logger.warn("[alter timeseries] device({}) alignedChunkMetadatas is null", device);
+      return;
+    }
+    // TODO To be optimized: Non-target modification measurements are directly written to data
+    List<IMeasurementSchema> schemaList =
+        collectSchemaList(alignedChunkMetadatas, reader, targetMeasurement, isTargetDevice);
+    List<IMeasurementSchema> schemaOldList =
+        collectSchemaList(alignedChunkMetadatas, reader, targetMeasurement, false);

Review Comment:
   These two steps result in two more IO operations before starting to rewrite, which can be combined into one step.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();

Review Comment:
   Add dataRegion write lock before closing all working processors to avoid writing operation from users later.



##########
server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewriteExcutor.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.alter;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.utils.CommonUtils;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.read.TsFileAlignedSeriesReaderIterator;
+import org.apache.iotdb.tsfile.read.TsFileDeviceIterator;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.IChunkReader;
+import org.apache.iotdb.tsfile.read.reader.IPointReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** This class is used to rewrite one tsfile with current encoding & compressionType. */
+public class TsFileRewriteExcutor {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteExcutor.class);
+
+  private final TsFileResource tsFileResource;
+  private final TsFileResource targetTsFileResource;
+  private final PartialPath fullPath;
+  private final TSEncoding curEncoding;
+  private final CompressionType curCompressionType;
+  private long timePartition;
+  private boolean sequence;
+  // record the min time and max time to update the target resource
+  private long minStartTimestamp = Long.MAX_VALUE;
+  private long maxEndTimestamp = Long.MIN_VALUE;
+
+  public TsFileRewriteExcutor(
+      TsFileResource tsFileResource,
+      TsFileResource targetTsFileResource,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence) {
+    this.tsFileResource = tsFileResource;
+    this.targetTsFileResource = targetTsFileResource;
+    this.fullPath = fullPath;
+    this.curEncoding = curEncoding;
+    this.curCompressionType = curCompressionType;
+    this.timePartition = timePartition;
+    this.sequence = sequence;
+  }
+
+  /** This function execute the rewrite task */
+  public void execute() throws IOException {
+
+    tsFileResource.tryReadLock();

Review Comment:
   If the resource is holding the write lock by another thread, this method will not get the read lock. Use `readLock()` instead.



##########
server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewriteExcutor.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.alter;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.utils.CommonUtils;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.read.TsFileAlignedSeriesReaderIterator;
+import org.apache.iotdb.tsfile.read.TsFileDeviceIterator;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.IChunkReader;
+import org.apache.iotdb.tsfile.read.reader.IPointReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** This class is used to rewrite one tsfile with current encoding & compressionType. */
+public class TsFileRewriteExcutor {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteExcutor.class);
+
+  private final TsFileResource tsFileResource;
+  private final TsFileResource targetTsFileResource;
+  private final PartialPath fullPath;
+  private final TSEncoding curEncoding;
+  private final CompressionType curCompressionType;
+  private long timePartition;
+  private boolean sequence;
+  // record the min time and max time to update the target resource
+  private long minStartTimestamp = Long.MAX_VALUE;
+  private long maxEndTimestamp = Long.MIN_VALUE;
+
+  public TsFileRewriteExcutor(
+      TsFileResource tsFileResource,
+      TsFileResource targetTsFileResource,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence) {
+    this.tsFileResource = tsFileResource;
+    this.targetTsFileResource = targetTsFileResource;
+    this.fullPath = fullPath;
+    this.curEncoding = curEncoding;
+    this.curCompressionType = curCompressionType;
+    this.timePartition = timePartition;
+    this.sequence = sequence;
+  }
+
+  /** This function execute the rewrite task */
+  public void execute() throws IOException {
+
+    tsFileResource.tryReadLock();
+    try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFilePath());
+        TsFileIOWriter writer = new TsFileIOWriter(targetTsFileResource.getTsFile())) {
+      // read devices
+      TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned();
+      while (deviceIterator.hasNext()) {
+        minStartTimestamp = Long.MAX_VALUE;
+        maxEndTimestamp = Long.MIN_VALUE;
+        Pair<String, Boolean> deviceInfo = deviceIterator.next();
+        String device = deviceInfo.left;
+        boolean aligned = deviceInfo.right;
+        // write chunkGroup header
+        writer.startChunkGroup(device);
+        boolean isTargetDevice = fullPath.getDevice().equals(device);
+        String targetMeasurement = fullPath.getMeasurement();
+        // write chunk & page data
+        if (aligned) {
+          rewriteAlgined(reader, writer, device, isTargetDevice, targetMeasurement);
+        } else {
+          rewriteNotAligned(device, reader, writer, targetMeasurement, isTargetDevice);
+        }
+        // chunkGroup end
+        writer.endChunkGroup();
+
+        targetTsFileResource.updateStartTime(device, minStartTimestamp);
+        targetTsFileResource.updateEndTime(device, maxEndTimestamp);
+      }
+
+      targetTsFileResource.updatePlanIndexes(tsFileResource);
+      // write index,bloom,footer, end file
+      writer.endFile();
+      targetTsFileResource.close();
+    } finally {
+      tsFileResource.readUnlock();
+    }
+  }
+
+  private void rewriteAlgined(
+      TsFileSequenceReader reader,
+      TsFileIOWriter writer,
+      String device,
+      boolean isTargetDevice,
+      String targetMeasurement)
+      throws IOException {
+    List<AlignedChunkMetadata> alignedChunkMetadatas = reader.getAlignedChunkMetadata(device);
+    if (alignedChunkMetadatas == null || alignedChunkMetadatas.isEmpty()) {
+      logger.warn("[alter timeseries] device({}) alignedChunkMetadatas is null", device);
+      return;
+    }
+    // TODO To be optimized: Non-target modification measurements are directly written to data
+    List<IMeasurementSchema> schemaList =
+        collectSchemaList(alignedChunkMetadatas, reader, targetMeasurement, isTargetDevice);
+    List<IMeasurementSchema> schemaOldList =
+        collectSchemaList(alignedChunkMetadatas, reader, targetMeasurement, false);
+    AlignedChunkWriterImpl chunkWriter = new AlignedChunkWriterImpl(schemaList);
+    TsFileAlignedSeriesReaderIterator readerIterator =
+        new TsFileAlignedSeriesReaderIterator(reader, alignedChunkMetadatas, schemaOldList);
+
+    while (readerIterator.hasNext()) {
+      Pair<AlignedChunkReader, Long> chunkReaderAndChunkSize = readerIterator.nextReader();
+      AlignedChunkReader chunkReader = chunkReaderAndChunkSize.left;
+      while (chunkReader.hasNextSatisfiedPage()) {
+        IBatchDataIterator batchDataIterator = chunkReader.nextPageData().getBatchDataIterator();
+        while (batchDataIterator.hasNext()) {
+          TsPrimitiveType[] pointsData = (TsPrimitiveType[]) batchDataIterator.currentValue();
+          long time = batchDataIterator.currentTime();
+          chunkWriter.write(time, pointsData);
+          targetTsFileResource.updateStartTime(device, time);
+          targetTsFileResource.updateEndTime(device, time);
+          batchDataIterator.next();
+        }
+      }
+    }
+    chunkWriter.writeToFileWriter(writer);
+  }
+
+  protected List<IMeasurementSchema> collectSchemaList(
+      List<AlignedChunkMetadata> alignedChunkMetadatas,
+      TsFileSequenceReader reader,
+      String targetMeasurement,
+      boolean isTargetDevice)
+      throws IOException {
+
+    Set<MeasurementSchema> schemaSet = new HashSet<>();
+    Set<String> measurementSet = new HashSet<>();

Review Comment:
   Use ArrayList to avoid type conversion later.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {

Review Comment:
   It seems that all TsFiles will be rewritten no matter it has corresponding device or not. Add a judgment to filter out tsfiles that do not contain the corresponding device.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)
+      throws IOException {
+
+    final String logKey =
+        logicalStorageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath();
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException(
+          "Alter failed. " + "Please do not delete until the old files upgraded.");
+    }
+    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
+      throw new IOException("Alter failed. " + "Please do not delete until the old files settled.");
+    }
+    logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey);
+    // flush & close
+    syncCloseAllWorkingTsFileProcessors();
+    logger.info("[alter timeseries] writeLock");
+    // wait lock
+    if (!tsFileManager.rewriteLockWithTimeout(
+        IoTDBDescriptor.getInstance().getConfig().getRewriteLockWaitTimeoutInMS())) {
+      throw new IOException(
+          "Alter failed. "
+              + "Other file rewriting operations are in progress, please do it later.");
+    }
+    // recover log
+    File logFile =
+        SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME);
+    if (logFile.exists()) {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      throw new IOException(
+          "Alter failed. "
+              + "alter.log detected, other alter operations may be running, please do it later.");
+    }
+    // rewrite target tsfiles
+    boolean done = false;
+    try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) {
+      Set<Long> timePartitions = tsFileManager.getTimePartitions();
+      // Record the ALTER process for server restart recovery
+      alteringLogger.logHeader(fullPath, curEncoding, curCompressionType, timePartitions);
+      for (Long timePartition : timePartitions) {
+        logger.info("[alter timeseries] {} alterDataInTsFiles seq({})", logKey, timePartition);
+        try {
+          rewriteDataInTsFiles(
+              tsFileManager.getSequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              true,
+              alteringLogger,
+              null,
+              logKey);
+          logger.info("[alter timeseries] {} alterDataInTsFiles unseq({})", logKey, timePartition);
+          rewriteDataInTsFiles(
+              tsFileManager.getUnsequenceListByTimePartition(timePartition),
+              fullPath,
+              curEncoding,
+              curCompressionType,
+              timePartition,
+              false,
+              alteringLogger,
+              null,
+              logKey);
+        } catch (IOException e) {
+          /**
+           * TODO If an exception occurs in the operation of a single tsfile, you need to terminate
+           * the operation and return 1. The modified schema and rewritten tsfile need to provide
+           * rollback commands 2. The tsfile that has not been rewritten needs to provide a recovery
+           * command
+           */
+          logger.error(
+              "[alter timeseries] " + logKey + " timePartition " + timePartition + " error", e);
+          throw e;
+        }
+      }
+      done = true;
+    } catch (Exception e) {
+      logger.error("[alter timeseries] " + logKey + " error", e);
+      throw e;
+    } finally {
+      logger.info("[alter timeseries] {} rewriteUnlock", logKey);
+      tsFileManager.rewriteUnlock();
+      // The process is complete and the logFile is deleted
+      if (done && logFile.exists()) {
+        FileUtils.delete(logFile);
+      }
+    }
+  }
+
+  private void rewriteDataInTsFiles(
+      List<TsFileResource> tsFileList,
+      PartialPath fullPath,
+      TSEncoding curEncoding,
+      CompressionType curCompressionType,
+      long timePartition,
+      boolean sequence,
+      AlteringLogger alteringLogger,
+      Set<TsFileIdentifier> undoneFiles,
+      String logKey)
+      throws IOException {
+
+    if (tsFileList == null || tsFileList.isEmpty()) {
+      return;
+    }
+    // log timePartition start
+    alteringLogger.startTimePartition(tsFileList, timePartition, sequence);
+    for (TsFileResource tsFileResource : tsFileList) {
+      if (!findUndoneResourcesAndRemove(tsFileResource, undoneFiles)) {
+        continue;
+      }
+      if (tsFileResource == null || !tsFileResource.isClosed()) {
+        return;
+      }
+      try {
+        logger.info(
+            "[alter timeseries] {} rewriteDataInTsFile:{}, fileSize:{} start",
+            logKey,
+            tsFileResource.getTsFilePath(),
+            tsFileResource.getTsFileSize());
+        // Generate the target tsFileResource
+        TsFileResource targetTsFileResource =
+            TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource);
+        // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file
+        TsFileRewriteExcutor tsFileRewriteExcutor =
+            new TsFileRewriteExcutor(
+                tsFileResource,
+                targetTsFileResource,
+                fullPath,
+                curEncoding,
+                curCompressionType,
+                timePartition,
+                sequence);
+        tsFileRewriteExcutor.execute();
+        // .tsfile->.alter.old .alter->.tsfile
+        logger.debug("[alter timeseries] {} move tsfile", logKey);
+        tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX);

Review Comment:
   add `resource.writeLock()` before moving.



##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -2069,6 +2249,235 @@ private List<TsFileResource> getFileResourceListForQuery(
     return tsfileResourcesForQuery;
   }
 
+  /**
+   * alter timeseries encoding & compressionType<br>
+   * 1、flush and close tsfile<br>
+   * 2、locks<br>
+   * 3、write temp tsfiles<br>
+   * 4、unregister old tsfiles and release locks<br>
+   * 5、rename temp tsfiles<br>
+   * 6、register tsfiles<br>
+   */
+  public void alter(
+      PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType)

Review Comment:
   What if fullPath is in the following format: root.sg.d1.**? Missing path judgment with wildcard characters.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTask.java:
##########
@@ -299,6 +299,11 @@ private boolean addReadLock(List<TsFileResource> tsFileResourceList) {
       return false;
     }
     try {
+      // Merge is not allowed when rewrite operation is running
+      if (tsFileManager.isRewriteLocked()) {
+        releaseAllLock();
+        return false;
+      }

Review Comment:
   Rewrite operations may get stuck on compaction for a long time, resulting in the accumulation of unseq data in the system.



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


Re: [PR] [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command [iotdb]

Posted by "lpf4254302 (via GitHub)" <gi...@apache.org>.
lpf4254302 closed pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command
URL: https://github.com/apache/iotdb/pull/6884


-- 
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] choubenson commented on a diff in pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

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


##########
server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java:
##########
@@ -3662,6 +4071,38 @@ public void setAllowCompaction(boolean allowCompaction) {
     this.tsFileManager.setAllowCompaction(allowCompaction);
   }
 
+  /** delete tsfile, copy from CompactionUtils TODO We need to rename CompactionUtils */
+  public static boolean deleteTsFile(TsFileResource seqFile, String storageGroupName) {
+    try {
+      logger.info("{} Start to delete TsFile {}", storageGroupName, seqFile.getTsFilePath());
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
+      //      seqFile.setStatus(TsFileResourceStatus.DELETED);
+      seqFile.delete();
+    } catch (IOException e) {
+      logger.error(e.getMessage(), e);
+      return false;
+    }
+    return true;
+  }
+
+  /** Delete all modification files for source files TODO We need to rename CompactionUtils */
+  public static void deleteModificationForSourceFile(
+      Collection<TsFileResource> sourceFiles, String storageGroupName) throws IOException {
+    logger.info("{} Start to delete modifications of source files", storageGroupName);
+    for (TsFileResource tsFileResource : sourceFiles) {
+      ModificationFile compactionModificationFile =
+          ModificationFile.getCompactionMods(tsFileResource);
+      if (compactionModificationFile.exists()) {
+        compactionModificationFile.remove();
+      }

Review Comment:
   It's unnecessary to delete compaction mods file, because compaction mods file is only produced during compaction.



-- 
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] lpf4254302 commented on pull request #6884: [IOTDB-3611] Support "Modify Time Series Encoding and Compression Type" interface/command

Posted by GitBox <gi...@apache.org>.
lpf4254302 commented on PR #6884:
URL: https://github.com/apache/iotdb/pull/6884#issuecomment-1221903369

   Phase III thinking modification code thinking:
   1. Modify the SCHEMA command-remove the rewriting code
   2. Modify the SCHEMA command-rewriting the log (here you only need to record the modification command record and the overwriting complete record)
   3. Modify the SCHEMA command-increase the rewriting memory record
   4. Modify the SCHEMA command-remove the original rewriting lock and use the new rewriting lock (only control the modification and finishing process, not the merger)
   5. Restore code-Remove all
   6. Restore code-increase the memory record through the log
   7. Consolidation code (inside and outside) -On the original rewriting lock
   8. Merge code (inside and outside) -writer's SCHEMA acquisition conditions: There is modification-> modification sequence-> query schema, otherwise use the file in the file
   9. Sort the command-file scan-whether there are modified sequences in the file, and the sequence coding has a list of changes.
   10. Sort out command-new rewriting lock control
   11. Sort out command-virtual storage group rewriting tasks
   12. Sort the command-Single file rewriting code modification
   13. Sort the command-alignment optimization


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