You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/03/18 02:07:43 UTC

[GitHub] [iotdb] neuyilan opened a new pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

neuyilan opened a new pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864


   ## Description
   
   
   ### Content1 ...
   
   ### Content2 ...
   
   ### Content3 ...
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error 
       conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, 
       design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design 
   (or naming) decision point and compare the alternatives with the designs that you've implemented 
   (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere 
   (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), 
   link to that discussion from this PR description and explain what have changed in your final design 
   compared to your original proposal or the consensus version in the end of the discussion. 
   If something hasn't changed since the original discussion, you can omit a detailed discussion of 
   those aspects of the design here, perhaps apart from brief mentioning for the sake of readability 
   of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   <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 or updated version, __license__, or notice information
   - [ ] 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.
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items 
   apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items 
   from the checklist above are strictly necessary, but it would be very helpful if you at least 
   self-review the PR. -->
   
   <hr>
   
   ##### Key changed/added classes (or packages if there are too many classes) in this PR
   


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

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605343179



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsfile(

Review comment:
       `rewriteTsfile` -> `rewriteTsFile`?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -18,92 +18,56 @@
  */
 package org.apache.iotdb.db.tools.upgrade;
 
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
-import org.apache.iotdb.tsfile.exception.write.PageException;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 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.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class TsFileOnlineUpgradeTool implements AutoCloseable {
+public class TsFileOnlineUpgradeTool extends TsFileRewriteTool implements AutoCloseable {

Review comment:
       Remove `implements AutoCloseable`?




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605484941



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -18,92 +18,56 @@
  */
 package org.apache.iotdb.db.tools.upgrade;
 
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
-import org.apache.iotdb.tsfile.exception.write.PageException;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 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.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class TsFileOnlineUpgradeTool implements AutoCloseable {
+public class TsFileOnlineUpgradeTool extends TsFileRewriteTool implements AutoCloseable {

Review comment:
       Yes, we can remove the AutoCloseable as it already extends TsFileRewriteTool.




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

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



[GitHub] [iotdb] neuyilan commented on pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#issuecomment-811581242


   @HTHou  PTAL 


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

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605484444



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -975,7 +981,23 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
 
-      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      try {
+        tsFileResource.getTimePartitionWithCheck();
+      } catch (PartitionViolationException e) {
+        logger.info("try to split the tsfile={}", tsFileResource.getTsFile().getPath());
+        TsFileRewriteTool.rewriteTsFile(tsFileResource, splitResources);

Review comment:
       I see, after we add a method to judgment whether it contains multiple partitions, then we do not need do the process logic in catch block.




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r606031495



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
##########
@@ -153,15 +153,25 @@ public long getTimePartition(String tsfilePath) {
   }
 
   @Override
-  public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
+  public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException {
     long startPartitionId = StorageEngine.getTimePartition(startTime);
     long endPartitionId = StorageEngine.getTimePartition(endTime);
     if (startPartitionId != endPartitionId) {
-      throw new PartitionViolationException(tsfilePath);
+      throw new PartitionViolationException(tsFilePath);
     }
     return startPartitionId;
   }
 
+  @Override
+  public boolean isSpanMultiTimePartitions(String tsFilePath) {
+    try {
+      getTimePartitionWithCheck(tsFilePath);
+    } catch (PartitionViolationException e) {
+      return false;

Review comment:
       Yes, thanks for your careful review, I agree with your opinion, the new commit here removed the business logic in the catch block and use -1 to indict the tsFile that may cross multiple time partitions. 
   As for `getTimePartition()` method ,  I prefer reserve the original two methods, one the `getTimePartition()` and the other is `getTimePartitionWithCheck() throws PartitionViolationException `, as almost method in the engine called is `getTimePartition()` method(Like the following picture shows, the method is called 37 times), the  `getTimePartitionWithCheck() throws PartitionViolationException` method only used when load one tsfile(Just 2 times ). If we combine the two method together such as `getTimePartition (string tsfilepath, Boolean needCheck) throws PartitionViolationException`, the caller may need to put the `needCheck` parameter all the time. it may be more complicated.
   What do you think?
   ![image](https://user-images.githubusercontent.com/6237070/113371019-d6526180-9397-11eb-8981-59846eb7c10d.png)
   ![image](https://user-images.githubusercontent.com/6237070/113371102-0e59a480-9398-11eb-8d59-5597af8e976e.png)
   




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

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r606026458



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
##########
@@ -153,15 +153,25 @@ public long getTimePartition(String tsfilePath) {
   }
 
   @Override
-  public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
+  public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException {
     long startPartitionId = StorageEngine.getTimePartition(startTime);
     long endPartitionId = StorageEngine.getTimePartition(endTime);
     if (startPartitionId != endPartitionId) {
-      throw new PartitionViolationException(tsfilePath);
+      throw new PartitionViolationException(tsFilePath);
     }
     return startPartitionId;
   }
 
+  @Override
+  public boolean isSpanMultiTimePartitions(String tsFilePath) {
+    try {
+      getTimePartitionWithCheck(tsFilePath);
+    } catch (PartitionViolationException e) {
+      return false;

Review comment:
       Yes, the  code above is just the sample code. What we most concerned about here is about removing the business logic in catch block. i.e., my suggestion is that  return `Pair<Boolean, Long> ` in `getTimePartitionWithCheck()` for remove the logic in catch block. About 'getTimePartition(..)',  both keep the original implementation or make a little improvement, such as' getTimePartition (string tsfilepath, Boolean needCheck) throws partitionviationexception `. I respect your opinion. What do you think? :)




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605493397



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsFile(
+      TsFileResource resourceToBeRewritten, List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    try (TsFileRewriteTool rewriteTool = new TsFileRewriteTool(resourceToBeRewritten)) {
+      rewriteTool.parseAndRewriteFile(rewrittenResources);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.reader.close();
+  }
+
+  /**
+   * Parse the old files and generate some new files according to the time partition interval.
+   *
+   * @throws IOException WriteProcessException
+   */
+  @SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning
+  public void parseAndRewriteFile(List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    // check if the TsFile has correct header
+    if (!fileCheck()) {
+      return;
+    }
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
+    reader.position(headerLength);
+    // start to scan chunks and chunkGroups
+    List<List<PageHeader>> pageHeadersInChunkGroup = new ArrayList<>();
+    List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
+    List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
+    byte marker;
+    List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+    String lastChunkGroupDeviceId = null;
+    try {
+      while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
+        switch (marker) {
+          case MetaMarker.CHUNK_HEADER:
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:

Review comment:
       In the new tsfile format, CHUNK_HEADER and ONLY_ONE_PAGE_CHUNK_HEADER both indicts the start of one chunk. @HTHou is an expert on tsfile format, please take a look at.




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605446958



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -975,7 +981,23 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
 
-      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      try {
+        tsFileResource.getTimePartitionWithCheck();

Review comment:
       That is another method, I will add another method to judge whether it contains multiple partitions. 




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

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



[GitHub] [iotdb] jixuan1989 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
jixuan1989 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605593290



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsFile(
+      TsFileResource resourceToBeRewritten, List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    try (TsFileRewriteTool rewriteTool = new TsFileRewriteTool(resourceToBeRewritten)) {
+      rewriteTool.parseAndRewriteFile(rewrittenResources);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.reader.close();
+  }
+
+  /**
+   * Parse the old files and generate some new files according to the time partition interval.
+   *
+   * @throws IOException WriteProcessException
+   */
+  @SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning
+  public void parseAndRewriteFile(List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    // check if the TsFile has correct header
+    if (!fileCheck()) {
+      return;
+    }
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
+    reader.position(headerLength);
+    // start to scan chunks and chunkGroups
+    List<List<PageHeader>> pageHeadersInChunkGroup = new ArrayList<>();
+    List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
+    List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
+    byte marker;
+    List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+    String lastChunkGroupDeviceId = null;
+    try {
+      while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
+        switch (marker) {
+          case MetaMarker.CHUNK_HEADER:
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:

Review comment:
       if "only one page in the chunk", then there is no pageStatistics in the page header. 
   
   Line 170 and 171 has indicated that in PageHeader's deserialize method: 
   ```
   PageHeader pageHeader =
                     reader.readPageHeader(dataType, header.getChunkType() == MetaMarker.CHUNK_HEADER);
   ```




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605575357



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
##########
@@ -153,15 +153,25 @@ public long getTimePartition(String tsfilePath) {
   }
 
   @Override
-  public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
+  public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException {
     long startPartitionId = StorageEngine.getTimePartition(startTime);
     long endPartitionId = StorageEngine.getTimePartition(endTime);
     if (startPartitionId != endPartitionId) {
-      throw new PartitionViolationException(tsfilePath);
+      throw new PartitionViolationException(tsFilePath);
     }
     return startPartitionId;
   }
 
+  @Override
+  public boolean isSpanMultiTimePartitions(String tsFilePath) {
+    try {
+      getTimePartitionWithCheck(tsFilePath);
+    } catch (PartitionViolationException e) {
+      return false;

Review comment:
       Hi,  I think the new `getTimePartition()` method will cost more time compare to the original one, because the new method above will iterate through all `deviceToIndex` arrays and calculate `deviceToIndex.size()` times to get the time partition id,  however in the original method, it will only calculate once. So I prefer to reserve the original method. And I have rearranged the code when checking or getting the time partition id, please take a look at it again.
   Thanks~




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605326191



##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileRead.java
##########
@@ -39,7 +39,8 @@
  * TsFileWriteWithTSRecord or TsFileWriteWithTablet to generate the test.tsfile first
  */
 public class TsFileRead {
-  private static final String DEVICE1 = "device_1";
+
+  private static final String DEVICE1 = Constant.STORAGE_GROUP + ".device_1";

Review comment:
       This is because in the TsFileWriteWithTablet class, the code write tsfile use root.sg info. if we write use the class and read use TsFileRead class, it will not work.
   So in this case, I will remove the storage info in TsFileWriteWithTablet class.




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

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r606026458



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
##########
@@ -153,15 +153,25 @@ public long getTimePartition(String tsfilePath) {
   }
 
   @Override
-  public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
+  public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException {
     long startPartitionId = StorageEngine.getTimePartition(startTime);
     long endPartitionId = StorageEngine.getTimePartition(endTime);
     if (startPartitionId != endPartitionId) {
-      throw new PartitionViolationException(tsfilePath);
+      throw new PartitionViolationException(tsFilePath);
     }
     return startPartitionId;
   }
 
+  @Override
+  public boolean isSpanMultiTimePartitions(String tsFilePath) {
+    try {
+      getTimePartitionWithCheck(tsFilePath);
+    } catch (PartitionViolationException e) {
+      return false;

Review comment:
       Yes, the  code above is just the sample code. What we most concerned about here is about removing the business logic in catch block. i.e., my suggestion is that  do not throw exception in `getTimePartitionWithCheck()` , we can return `Pair<Boolean, Long> ` or I notice that your new commit return -1 as the cross partitions.
   About 'getTimePartition(..)',  both keep the original implementation or make a little improvement, such as' getTimePartition (string tsfilepath, Boolean needCheck) throws partitionviationexception `. I respect your opinion. What do you think? :)




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

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



[GitHub] [iotdb] mychaow merged pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
mychaow merged pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864


   


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

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



[GitHub] [iotdb] HTHou commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r601134034



##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileRead.java
##########
@@ -39,7 +39,8 @@
  * TsFileWriteWithTSRecord or TsFileWriteWithTablet to generate the test.tsfile first
  */
 public class TsFileRead {
-  private static final String DEVICE1 = "device_1";
+
+  private static final String DEVICE1 = Constant.STORAGE_GROUP + ".device_1";

Review comment:
       Actually it's not necessary to indicate the storage group for TsFile module. The storage group is a concept of IoTDB only. We are supporting to use TsFile module stand alone and this example is for this purpose. 




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605445842



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -975,7 +981,23 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
 
-      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      try {
+        tsFileResource.getTimePartitionWithCheck();
+      } catch (PartitionViolationException e) {
+        logger.info("try to split the tsfile={}", tsFileResource.getTsFile().getPath());
+        TsFileRewriteTool.rewriteTsFile(tsFileResource, splitResources);

Review comment:
       The purpose here is to split and rewrite a tsFile only when it spans multiple partitions. When the tsFile is not partitioned across time, there is no need to split and rewrite it. Otherwise, we may need a function to determine whether the tsFile spans multiple time partitions. But that's exactly what the `getTimePartitionWithCheck()` function does, so I set the split logic 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.

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605482267



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -152,7 +111,7 @@ private void upgradeFile(List<TsFileResource> upgradedResources)
             if (newChunkGroup) {

Review comment:
       I see this code is not used, I think this is legacy code, I will remove this variable. Thanks.




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605358860



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsfile(

Review comment:
       Thanks, solved




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605326191



##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileRead.java
##########
@@ -39,7 +39,8 @@
  * TsFileWriteWithTSRecord or TsFileWriteWithTablet to generate the test.tsfile first
  */
 public class TsFileRead {
-  private static final String DEVICE1 = "device_1";
+
+  private static final String DEVICE1 = Constant.STORAGE_GROUP + ".device_1";

Review comment:
       This is because in the TsFileWriteWithTablet class, the code write tsfile use root.sg info. if we write use the class and read use TsFileRead class, it will not work.
   So in this case, I will remove the storage group info in TsFileWriteWithTablet class.




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

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



[GitHub] [iotdb] neuyilan commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605484596



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -18,92 +18,56 @@
  */
 package org.apache.iotdb.db.tools.upgrade;
 
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
-import org.apache.iotdb.tsfile.exception.write.PageException;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 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.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class TsFileOnlineUpgradeTool implements AutoCloseable {
+public class TsFileOnlineUpgradeTool extends TsFileRewriteTool implements AutoCloseable {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileOnlineUpgradeTool.class);
 
-  private TsFileSequenceReaderForV2 reader;
-  private File oldTsFile;
-  private List<Modification> oldModification;
-  private Iterator<Modification> modsIterator;
-  // new tsFile writer -> list of new modification
-  private Map<TsFileIOWriter, ModificationFile> fileModificationMap;
-  private Deletion currentMod;
-  private Decoder defaultTimeDecoder =
-      Decoder.getDecoderByType(
-          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
-          TSDataType.INT64);
-  private Decoder valueDecoder;
-
-  // PartitionId -> TsFileIOWriter
-  private Map<Long, TsFileIOWriter> partitionWriterMap;
-
   /**
-   * Create a file reader of the given file. The reader will read the tail of the file to get the
-   * file metadata size. Then the reader will skip the first TSFileConfig.OLD_MAGIC_STRING.length()
-   * bytes of the file for preparing reading real data.
+   * Create a file reader of the given file. This reader will read the old file and rewrite it to a
+   * new format(v3) file
    *
-   * @param file the data file
+   * @param resourceToBeUpgraded the old tsfile resource which need to be upgrade
    * @throws IOException If some I/O error occurs
    */
   public TsFileOnlineUpgradeTool(TsFileResource resourceToBeUpgraded) throws IOException {
-    oldTsFile = resourceToBeUpgraded.getTsFile();
+    super(resourceToBeUpgraded);
     String file = oldTsFile.getAbsolutePath();
     reader = new TsFileSequenceReaderForV2(file);
-    partitionWriterMap = new HashMap<>();
-    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
-      oldModification = (List<Modification>) resourceToBeUpgraded.getModFile().getModifications();
-      modsIterator = oldModification.iterator();
-      fileModificationMap = new HashMap<>();
-    }
   }
 
   /**
-   * upgrade a single TsFile
+   * upgrade a single TsFile.
    *
-   * @param tsFileName old version tsFile's absolute path
+   * @param resourceToBeUpgraded the old file's resource which need to be upgrade.
    * @param upgradedResources new version tsFiles' resources
    */
   public static void upgradeOneTsfile(

Review comment:
       Thanks, done




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

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



[GitHub] [iotdb] jixuan1989 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
jixuan1989 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605593290



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsFile(
+      TsFileResource resourceToBeRewritten, List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    try (TsFileRewriteTool rewriteTool = new TsFileRewriteTool(resourceToBeRewritten)) {
+      rewriteTool.parseAndRewriteFile(rewrittenResources);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.reader.close();
+  }
+
+  /**
+   * Parse the old files and generate some new files according to the time partition interval.
+   *
+   * @throws IOException WriteProcessException
+   */
+  @SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning
+  public void parseAndRewriteFile(List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    // check if the TsFile has correct header
+    if (!fileCheck()) {
+      return;
+    }
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
+    reader.position(headerLength);
+    // start to scan chunks and chunkGroups
+    List<List<PageHeader>> pageHeadersInChunkGroup = new ArrayList<>();
+    List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
+    List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
+    byte marker;
+    List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+    String lastChunkGroupDeviceId = null;
+    try {
+      while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
+        switch (marker) {
+          case MetaMarker.CHUNK_HEADER:
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:

Review comment:
       if "only one page in the chunk", then there is no pageStatistics in the page header. 
   
   Line 170 and 171 has indicated that in PageHeader's deserialize method: 
   ```
   PageHeader pageHeader =
                     reader.readPageHeader(dataType, header.getChunkType() == MetaMarker.CHUNK_HEADER);
   ```
   
   So, it is correct that using the same logic for case `CHUNK_HEADER` and `ONLY_ONE_PAGE_CHUNK_HEADER` 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.

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605349478



##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -18,92 +18,56 @@
  */
 package org.apache.iotdb.db.tools.upgrade;
 
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
-import org.apache.iotdb.tsfile.exception.write.PageException;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 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.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class TsFileOnlineUpgradeTool implements AutoCloseable {
+public class TsFileOnlineUpgradeTool extends TsFileRewriteTool implements AutoCloseable {

Review comment:
       Remove `implements AutoCloseable`?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -18,92 +18,56 @@
  */
 package org.apache.iotdb.db.tools.upgrade;
 
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
-import org.apache.iotdb.tsfile.exception.write.PageException;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
-import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 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.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class TsFileOnlineUpgradeTool implements AutoCloseable {
+public class TsFileOnlineUpgradeTool extends TsFileRewriteTool implements AutoCloseable {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileOnlineUpgradeTool.class);
 
-  private TsFileSequenceReaderForV2 reader;
-  private File oldTsFile;
-  private List<Modification> oldModification;
-  private Iterator<Modification> modsIterator;
-  // new tsFile writer -> list of new modification
-  private Map<TsFileIOWriter, ModificationFile> fileModificationMap;
-  private Deletion currentMod;
-  private Decoder defaultTimeDecoder =
-      Decoder.getDecoderByType(
-          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
-          TSDataType.INT64);
-  private Decoder valueDecoder;
-
-  // PartitionId -> TsFileIOWriter
-  private Map<Long, TsFileIOWriter> partitionWriterMap;
-
   /**
-   * Create a file reader of the given file. The reader will read the tail of the file to get the
-   * file metadata size. Then the reader will skip the first TSFileConfig.OLD_MAGIC_STRING.length()
-   * bytes of the file for preparing reading real data.
+   * Create a file reader of the given file. This reader will read the old file and rewrite it to a
+   * new format(v3) file
    *
-   * @param file the data file
+   * @param resourceToBeUpgraded the old tsfile resource which need to be upgrade
    * @throws IOException If some I/O error occurs
    */
   public TsFileOnlineUpgradeTool(TsFileResource resourceToBeUpgraded) throws IOException {
-    oldTsFile = resourceToBeUpgraded.getTsFile();
+    super(resourceToBeUpgraded);
     String file = oldTsFile.getAbsolutePath();
     reader = new TsFileSequenceReaderForV2(file);
-    partitionWriterMap = new HashMap<>();
-    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
-      oldModification = (List<Modification>) resourceToBeUpgraded.getModFile().getModifications();
-      modsIterator = oldModification.iterator();
-      fileModificationMap = new HashMap<>();
-    }
   }
 
   /**
-   * upgrade a single TsFile
+   * upgrade a single TsFile.
    *
-   * @param tsFileName old version tsFile's absolute path
+   * @param resourceToBeUpgraded the old file's resource which need to be upgrade.
    * @param upgradedResources new version tsFiles' resources
    */
   public static void upgradeOneTsfile(

Review comment:
       `upgradeOneTsfile` -> `upgradeOneTsFile`?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
##########
@@ -244,24 +244,19 @@ public long getTimePartition(String tsfilePath) {
   @Override
   public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
     long partitionId = -1;
-    for (Long startTime : startTimes) {
-      long p = StorageEngine.getTimePartition(startTime);
+    for (int index : deviceToIndex.values()) {

Review comment:
       nice change!

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.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.tools;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+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.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class TsFileRewriteTool implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileRewriteTool.class);
+
+  protected TsFileSequenceReader reader;
+  protected File oldTsFile;
+  protected List<Modification> oldModification;
+  protected Iterator<Modification> modsIterator;
+
+  /** new tsFile writer -> list of new modification */
+  protected Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+
+  protected Deletion currentMod;
+  protected Decoder defaultTimeDecoder =
+      Decoder.getDecoderByType(
+          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+          TSDataType.INT64);
+  protected Decoder valueDecoder;
+
+  /** PartitionId -> TsFileIOWriter */
+  protected Map<Long, TsFileIOWriter> partitionWriterMap;
+
+  /** Maximum index of plans executed within this TsFile. */
+  protected long maxPlanIndex = Long.MIN_VALUE;
+
+  /** Minimum index of plans executed within this TsFile. */
+  protected long minPlanIndex = Long.MAX_VALUE;
+
+  /**
+   * Create a file reader of the given file. The reader will read the real data and rewrite to some
+   * new tsFiles.
+   *
+   * @throws IOException If some I/O error occurs
+   */
+  public TsFileRewriteTool(TsFileResource resourceToBeRewritten) throws IOException {
+    oldTsFile = resourceToBeRewritten.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReader(file);
+    partitionWriterMap = new HashMap<>();
+    if (FSFactoryProducer.getFSFactory().getFile(file + ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeRewritten.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
+    }
+  }
+
+  /**
+   * Rewrite an old file to the latest version
+   *
+   * @param resourceToBeRewritten the tsfile which to be rewrite
+   * @param rewrittenResources the rewritten files
+   */
+  public static void rewriteTsFile(
+      TsFileResource resourceToBeRewritten, List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    try (TsFileRewriteTool rewriteTool = new TsFileRewriteTool(resourceToBeRewritten)) {
+      rewriteTool.parseAndRewriteFile(rewrittenResources);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.reader.close();
+  }
+
+  /**
+   * Parse the old files and generate some new files according to the time partition interval.
+   *
+   * @throws IOException WriteProcessException
+   */
+  @SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning
+  public void parseAndRewriteFile(List<TsFileResource> rewrittenResources)
+      throws IOException, WriteProcessException {
+    // check if the TsFile has correct header
+    if (!fileCheck()) {
+      return;
+    }
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
+    reader.position(headerLength);
+    // start to scan chunks and chunkGroups
+    List<List<PageHeader>> pageHeadersInChunkGroup = new ArrayList<>();
+    List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
+    List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
+    byte marker;
+    List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+    String lastChunkGroupDeviceId = null;
+    try {
+      while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
+        switch (marker) {
+          case MetaMarker.CHUNK_HEADER:
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:

Review comment:
       Not sure, do we need deal with `CHUNK_HEADER`?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
##########
@@ -244,24 +244,19 @@ public long getTimePartition(String tsfilePath) {
   @Override
   public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {

Review comment:
       `tsfilePath` -> `tsFilePath`?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -975,7 +981,23 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
 
-      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      try {
+        tsFileResource.getTimePartitionWithCheck();

Review comment:
       How about we add a method that  judgment  whether it contains multiple partitions?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -975,7 +981,23 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
 
-      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      try {
+        tsFileResource.getTimePartitionWithCheck();
+      } catch (PartitionViolationException e) {
+        logger.info("try to split the tsfile={}", tsFileResource.getTsFile().getPath());
+        TsFileRewriteTool.rewriteTsFile(tsFileResource, splitResources);

Review comment:
       Would be great if we rewrite the tsFile not in the catch block. What do you think?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -152,7 +111,7 @@ private void upgradeFile(List<TsFileResource> upgradedResources)
             if (newChunkGroup) {

Review comment:
       Why do we need to check if it's a new chunk?i.e., Where is the purpose of this variable check? 




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

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



[GitHub] [iotdb] neuyilan edited a comment on pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan edited a comment on pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#issuecomment-812279896


   > Our common goal is to make iotdb better
   
   @sunjincheng121 
   Thank you for your sincere suggestions. Your suggestions are very good. Yes, we all hope iotdb will be better and better :).
   BTW, I've unified the vocabulary to use spans time partition to represent one tsfile spans multiple time partitions.
   Thank you again for your careful review.


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

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



[GitHub] [iotdb] neuyilan commented on pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
neuyilan commented on pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#issuecomment-812279896


   > Our common goal is to make iotdb better
   @sunjincheng121 
   Thank you for your sincere suggestions. Your suggestions are very good. Yes, we all hope iotdb will be better and better :).
   BTW, I've unified the vocabulary to use spans time partition to represent one tsfile spans multiple time partitions.
   Thank you again for your careful review.


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

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



[GitHub] [iotdb] sunjincheng121 commented on a change in pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on a change in pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#discussion_r605539025



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
##########
@@ -153,15 +153,25 @@ public long getTimePartition(String tsfilePath) {
   }
 
   @Override
-  public long getTimePartitionWithCheck(String tsfilePath) throws PartitionViolationException {
+  public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException {
     long startPartitionId = StorageEngine.getTimePartition(startTime);
     long endPartitionId = StorageEngine.getTimePartition(endTime);
     if (startPartitionId != endPartitionId) {
-      throw new PartitionViolationException(tsfilePath);
+      throw new PartitionViolationException(tsFilePath);
     }
     return startPartitionId;
   }
 
+  @Override
+  public boolean isSpanMultiTimePartitions(String tsFilePath) {
+    try {
+      getTimePartitionWithCheck(tsFilePath);
+    } catch (PartitionViolationException e) {
+      return false;

Review comment:
       Frankly, I don't recommend returning method values in a catch block. :)
   I have an idea to do the following:
   
   ```
     @Override
     public long getTimePartition(String tsFilePath) throws PartitionViolationException {
       Pair<Boolean, Long> pair = getTimePartitionWithCheck();
       if(pair.left) {
         throw new PartitionViolationException(tsFilePath);
       }else {
         return pair.right;
       }
     }
   
     public Pair<Boolean, Long> getTimePartitionWithCheck() {
       Pair<Boolean, Long> pair = new Pair(true, -1);
       for (int index : deviceToIndex.values()) {
         long p = StorageEngine.getTimePartition(startTimes[index]);
         if ( pair.right == -1) {
           pair.right = p;
         } else {
           if ( pair.right != p) {
             return pair;
           }
         }
         p = StorageEngine.getTimePartition(endTimes[index]);
         if ( pair.right != p) {
           return pair;
         }
       }
       if ( pair.right == -1) {
         return pair;
       }
       pair.left = false;
       return pair;
     }
   ```
   
   Of course,I know that there is already an implementation of 'gettimepartition(..)'. We can take advantage of this opportunity to refactor it. What do you think?




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

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



[GitHub] [iotdb] sunjincheng121 commented on pull request #2864: [IOTDB-1230]Support cross multi time partition when loading one TsFile

Posted by GitBox <gi...@apache.org>.
sunjincheng121 commented on pull request #2864:
URL: https://github.com/apache/iotdb/pull/2864#issuecomment-812284132


   The PR will be merged after the CI completed...


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

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