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/05/26 15:30:05 UTC

[GitHub] [iotdb] THUMarkLau opened a new pull request, #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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

   Currently, in the compaction process, the metadata information of time series needs to be obtained from the MetadataManager. This can be a performance bottleneck in distributed edition in the future. Therefore, this Issue aims to remove the access to the MetadataManager and get the time series metadata information directly from the TsFile.


-- 
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] THUMarkLau commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java:
##########
@@ -201,8 +201,10 @@ protected void doCompaction() throws Exception {
     } catch (Throwable throwable) {
       // catch throwable to handle OOM errors
       if (!(throwable instanceof InterruptedException)) {
-        LOGGER.error(
-            "{} [Compaction] Meet errors in inner space compaction.", fullStorageGroupName);
+        //        LOGGER.error(
+        //            "{} [Compaction] Meet errors in inner space compaction.",
+        // fullStorageGroupName);
+        throwable.printStackTrace();

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] THUMarkLau commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/ReadPointPerformerSubTask.java:
##########
@@ -61,30 +59,22 @@ public ReadPointPerformerSubTask(
       QueryContext queryContext,
       QueryDataSource queryDataSource,
       AbstractCompactionWriter compactionWriter,
+      Map<String, MeasurementSchema> schemaMap,
       int taskId) {
     this.device = device;
     this.measurementList = measurementList;
     this.queryContext = queryContext;
     this.queryDataSource = queryDataSource;
     this.compactionWriter = compactionWriter;
+    this.schemaMap = schemaMap;
     this.taskId = taskId;
   }
 
   @Override
   public Void call() throws Exception {
     for (String measurement : measurementList) {
       List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
-      try {
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
-          measurementSchemas.add(IDTableManager.getInstance().getSeriesSchema(device, measurement));
-        } else {
-          measurementSchemas.add(
-              IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
-        }
-      } catch (PathNotExistException e) {
-        logger.info("A deleted path is skipped: {}", e.getMessage());
-        continue;
-      }
+      measurementSchemas.add(schemaMap.get(measurement));

Review Comment:
   resolve



##########
server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java:
##########
@@ -337,7 +339,9 @@ public void testRewriteCrossCompactionFileStatus() throws Exception {
     }
 
     CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
-    CompactionTaskManager.getInstance().waitAllCompactionFinish();
+    while (!task.isTaskFinished()) {
+      TimeUnit.MILLISECONDS.sleep(200);
+    }

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] qiaojialin merged pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


-- 
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 #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -239,6 +238,60 @@ private void compactNonAlignedSeries(
     compactionWriter.endChunkGroup();
   }
 
+  private ConcurrentHashMap<String, MeasurementSchema> getMeasurementSchema(
+      String device, Set<String> measurements) throws IllegalPathException, IOException {
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap = new ConcurrentHashMap<>();
+    List<TsFileResource> allResources = new LinkedList<>(seqFiles);
+    allResources.addAll(unseqFiles);
+    for (String measurement : measurements) {
+      for (TsFileResource tsFileResource : allResources) {
+        if (!tsFileResource.mayContainsDevice(device)) {
+          continue;
+        }
+        MeasurementSchema schema =
+            getMeasurementSchemaFromReader(
+                readerCacheMap.computeIfAbsent(
+                    tsFileResource,
+                    x -> {
+                      try {
+                        return new TsFileSequenceReader(x.getTsFilePath());

Review Comment:
   Use FileReaderManager to get TsFileSequenceReader to avoid "too many open files" exception.



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

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

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


[GitHub] [iotdb] qiaojialin commented on pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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

   This PR could be updated after separating data with index in TsFile


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

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

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


[GitHub] [iotdb] qiaojialin commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java:
##########
@@ -201,8 +201,10 @@ protected void doCompaction() throws Exception {
     } catch (Throwable throwable) {
       // catch throwable to handle OOM errors
       if (!(throwable instanceof InterruptedException)) {
-        LOGGER.error(
-            "{} [Compaction] Meet errors in inner space compaction.", fullStorageGroupName);
+        //        LOGGER.error(
+        //            "{} [Compaction] Meet errors in inner space compaction.",
+        // fullStorageGroupName);

Review Comment:
   remove



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java:
##########
@@ -201,8 +201,10 @@ protected void doCompaction() throws Exception {
     } catch (Throwable throwable) {
       // catch throwable to handle OOM errors
       if (!(throwable instanceof InterruptedException)) {
-        LOGGER.error(
-            "{} [Compaction] Meet errors in inner space compaction.", fullStorageGroupName);
+        //        LOGGER.error(
+        //            "{} [Compaction] Meet errors in inner space compaction.",
+        // fullStorageGroupName);
+        throwable.printStackTrace();

Review Comment:
   use logger if you want to print



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -237,6 +236,63 @@ private void compactNonAlignedSeries(
     compactionWriter.endChunkGroup();
   }
 
+  private Map<String, MeasurementSchema> getMeasurementSchema(
+      String device, Set<String> measurements) throws IllegalPathException, IOException {
+    HashMap<String, MeasurementSchema> schemaMap = new HashMap<>();
+    List<TsFileResource> allResources = new LinkedList<>(seqFiles);
+    allResources.addAll(unseqFiles);
+    for (String measurement : measurements) {
+      for (TsFileResource tsFileResource : allResources) {

Review Comment:
   better to scan from the last, a previous resource may have a deleted chunk with different datatype



-- 
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] jt2594838 commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java:
##########
@@ -386,11 +386,20 @@ public void restart() throws InterruptedException {
         this.taskExecutionPool.shutdownNow();
         this.taskExecutionPool.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
       }
+      if (this.subCompactionTaskExecutionPool != null) {
+        subCompactionTaskExecutionPool.shutdownNow();
+        this.subCompactionTaskExecutionPool.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+      }

Review Comment:
   Waiting forever is not recommended for the sake of debugging. Better to change it to a timed wait and print a log or throw an exception to the upper level.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/SingleSeriesCompactionExecutor.java:
##########
@@ -101,6 +121,9 @@ public void execute() throws IOException {
       List<ChunkMetadata> chunkMetadataList = readerListPair.right;
       for (ChunkMetadata chunkMetadata : chunkMetadataList) {
         Chunk currentChunk = reader.readMemChunk(chunkMetadata);
+        if (this.chunkWriter == null) {
+          constructChunkWriterFromReadChunk(currentChunk);
+        }

Review Comment:
   In the write-whole-chunk path, the chunk writer seems not used (FileWriter is used instead). So maybe it is too early to initialize it here. Maybe you could add a getter of chunkWriter and let the getter do the initialization.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/AlignedSeriesCompactionExecutor.java:
##########
@@ -99,9 +105,18 @@ private List<IMeasurementSchema> collectSchemaFromAlignedChunkMetadataList(
           if (chunkMetadata == null) {
             continue;
           }
+          if (measurementSet.contains(chunkMetadata.getMeasurementUid())) {
+            continue;
+          }
+          measurementSet.add(chunkMetadata.getMeasurementUid());
+          Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata);
+          ChunkHeader header = chunk.getHeader();
           schemaSet.add(
               new MeasurementSchema(
-                  chunkMetadata.getMeasurementUid(), chunkMetadata.getDataType()));
+                  header.getMeasurementID(),
+                  header.getDataType(),
+                  header.getEncodingType(),
+                  header.getCompressionType()));

Review Comment:
   The chunk's data seems to be wasted. Is there any chance to reuse it? Maybe you can retrieve it from the chunk cache.



##########
server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java:
##########
@@ -337,7 +339,9 @@ public void testRewriteCrossCompactionFileStatus() throws Exception {
     }
 
     CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
-    CompactionTaskManager.getInstance().waitAllCompactionFinish();
+    while (!task.isTaskFinished()) {
+      TimeUnit.MILLISECONDS.sleep(200);
+    }

Review Comment:
   Add a failure after some timeout to avoid the test being blocked forever.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -239,6 +238,60 @@ private void compactNonAlignedSeries(
     compactionWriter.endChunkGroup();
   }
 
+  private ConcurrentHashMap<String, MeasurementSchema> getMeasurementSchema(
+      String device, Set<String> measurements) throws IllegalPathException, IOException {
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap = new ConcurrentHashMap<>();
+    List<TsFileResource> allResources = new LinkedList<>(seqFiles);
+    allResources.addAll(unseqFiles);
+    for (String measurement : measurements) {
+      for (TsFileResource tsFileResource : allResources) {
+        if (!tsFileResource.mayContainsDevice(device)) {
+          continue;
+        }
+        MeasurementSchema schema =
+            getMeasurementSchemaFromReader(
+                readerCacheMap.computeIfAbsent(
+                    tsFileResource,
+                    x -> {
+                      try {
+                        return new TsFileSequenceReader(x.getTsFilePath());
+                      } catch (IOException e) {
+                        throw new RuntimeException(
+                            String.format(
+                                "Failed to construct sequence reader for %s", tsFileResource));
+                      }
+                    }),

Review Comment:
   Throwing a runtime exception is not a very delicate way. You may catch it using an AtomicReference or something else and throw it outside.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -192,19 +189,20 @@ private void compactNonAlignedSeries(
       AbstractCompactionWriter compactionWriter,
       QueryContext queryContext,
       QueryDataSource queryDataSource)
-      throws MetadataException, IOException, InterruptedException {
-    boolean hasStartChunkGroup = false;
+      throws IOException, InterruptedException, IllegalPathException {
     MultiTsFileDeviceIterator.MeasurementIterator measurementIterator =
         deviceIterator.iterateNotAlignedSeries(device, false);
     Set<String> allMeasurements = measurementIterator.getAllMeasurements();
     int subTaskNums = Math.min(allMeasurements.size(), subTaskNum);
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap =
+        getMeasurementSchema(device, allMeasurements);
 
     // assign all measurements to different sub tasks
     Set<String>[] measurementsForEachSubTask = new HashSet[subTaskNums];
     int idx = 0;
     for (String measurement : allMeasurements) {
       if (measurementsForEachSubTask[idx % subTaskNums] == null) {
-        measurementsForEachSubTask[idx % subTaskNums] = new HashSet<String>();
+        measurementsForEachSubTask[idx % subTaskNums] = new HashSet<>();
       }
       measurementsForEachSubTask[idx++ % subTaskNums].add(measurement);
     }

Review Comment:
   Is it necessary to use sets, why is it not enough to use lists?



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -146,19 +153,9 @@ private void compactAlignedSeries(
     MultiTsFileDeviceIterator.AlignedMeasurementIterator alignedMeasurementIterator =
         deviceIterator.iterateAlignedSeries(device);
     Set<String> allMeasurements = alignedMeasurementIterator.getAllMeasurements();
-    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
-    for (String measurement : allMeasurements) {
-      try {
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
-          measurementSchemas.add(IDTableManager.getInstance().getSeriesSchema(device, measurement));
-        } else {
-          measurementSchemas.add(
-              IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
-        }
-      } catch (PathNotExistException e) {
-        LOGGER.info("A deleted path is skipped: {}", e.getMessage());
-      }
-    }
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap =
+        getMeasurementSchema(device, allMeasurements);
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>(schemaMap.values());

Review Comment:
   The map is read-only after initialization, so it does not seem necessary to use a concurrent one.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/ReadPointPerformerSubTask.java:
##########
@@ -61,30 +59,22 @@ public ReadPointPerformerSubTask(
       QueryContext queryContext,
       QueryDataSource queryDataSource,
       AbstractCompactionWriter compactionWriter,
+      Map<String, MeasurementSchema> schemaMap,
       int taskId) {
     this.device = device;
     this.measurementList = measurementList;
     this.queryContext = queryContext;
     this.queryDataSource = queryDataSource;
     this.compactionWriter = compactionWriter;
+    this.schemaMap = schemaMap;
     this.taskId = taskId;
   }
 
   @Override
   public Void call() throws Exception {
     for (String measurement : measurementList) {
       List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
-      try {
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
-          measurementSchemas.add(IDTableManager.getInstance().getSeriesSchema(device, measurement));
-        } else {
-          measurementSchemas.add(
-              IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
-        }
-      } catch (PathNotExistException e) {
-        logger.info("A deleted path is skipped: {}", e.getMessage());
-        continue;
-      }
+      measurementSchemas.add(schemaMap.get(measurement));

Review Comment:
   How about using a singleton list?



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -239,6 +238,60 @@ private void compactNonAlignedSeries(
     compactionWriter.endChunkGroup();
   }
 
+  private ConcurrentHashMap<String, MeasurementSchema> getMeasurementSchema(
+      String device, Set<String> measurements) throws IllegalPathException, IOException {
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap = new ConcurrentHashMap<>();
+    List<TsFileResource> allResources = new LinkedList<>(seqFiles);
+    allResources.addAll(unseqFiles);
+    for (String measurement : measurements) {
+      for (TsFileResource tsFileResource : allResources) {
+        if (!tsFileResource.mayContainsDevice(device)) {
+          continue;
+        }
+        MeasurementSchema schema =
+            getMeasurementSchemaFromReader(
+                readerCacheMap.computeIfAbsent(
+                    tsFileResource,
+                    x -> {
+                      try {
+                        return new TsFileSequenceReader(x.getTsFilePath());
+                      } catch (IOException e) {
+                        throw new RuntimeException(
+                            String.format(
+                                "Failed to construct sequence reader for %s", tsFileResource));
+                      }
+                    }),
+                device,
+                measurement);
+        if (schema != null) {
+          schemaMap.put(measurement, schema);
+          break;
+        }
+      }
+    }
+    return schemaMap;
+  }
+
+  private MeasurementSchema getMeasurementSchemaFromReader(
+      TsFileSequenceReader reader, String device, String measurement)
+      throws IllegalPathException, IOException {
+    List<ChunkMetadata> chunkMetadata =
+        reader.getChunkMetadataList(new PartialPath(device, measurement));
+    if (chunkMetadata.size() > 0) {
+      Chunk chunk = reader.readMemChunk(chunkMetadata.get(0));
+      ChunkHeader header = chunk.getHeader();
+      return new MeasurementSchema(
+          measurement, header.getDataType(), header.getEncodingType(), header.getCompressionType());
+    }
+    return null;
+  }

Review Comment:
   Reading all ChunkMetadata and the one chunk just for the MeasurementSchema seems too costly, and I think the design should be revised.



-- 
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] THUMarkLau commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java:
##########
@@ -386,11 +386,20 @@ public void restart() throws InterruptedException {
         this.taskExecutionPool.shutdownNow();
         this.taskExecutionPool.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
       }
+      if (this.subCompactionTaskExecutionPool != null) {
+        subCompactionTaskExecutionPool.shutdownNow();
+        this.subCompactionTaskExecutionPool.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+      }

Review Comment:
   resolve



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/AlignedSeriesCompactionExecutor.java:
##########
@@ -99,9 +105,18 @@ private List<IMeasurementSchema> collectSchemaFromAlignedChunkMetadataList(
           if (chunkMetadata == null) {
             continue;
           }
+          if (measurementSet.contains(chunkMetadata.getMeasurementUid())) {
+            continue;
+          }
+          measurementSet.add(chunkMetadata.getMeasurementUid());
+          Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata);
+          ChunkHeader header = chunk.getHeader();
           schemaSet.add(
               new MeasurementSchema(
-                  chunkMetadata.getMeasurementUid(), chunkMetadata.getDataType()));
+                  header.getMeasurementID(),
+                  header.getDataType(),
+                  header.getEncodingType(),
+                  header.getCompressionType()));

Review Comment:
   resolve



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -146,19 +153,9 @@ private void compactAlignedSeries(
     MultiTsFileDeviceIterator.AlignedMeasurementIterator alignedMeasurementIterator =
         deviceIterator.iterateAlignedSeries(device);
     Set<String> allMeasurements = alignedMeasurementIterator.getAllMeasurements();
-    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
-    for (String measurement : allMeasurements) {
-      try {
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
-          measurementSchemas.add(IDTableManager.getInstance().getSeriesSchema(device, measurement));
-        } else {
-          measurementSchemas.add(
-              IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
-        }
-      } catch (PathNotExistException e) {
-        LOGGER.info("A deleted path is skipped: {}", e.getMessage());
-      }
-    }
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap =
+        getMeasurementSchema(device, allMeasurements);
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>(schemaMap.values());

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] THUMarkLau closed pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

Posted by GitBox <gi...@apache.org>.
THUMarkLau closed pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction
URL: https://github.com/apache/iotdb/pull/5572


-- 
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] THUMarkLau commented on a diff in pull request #5572: [IOTDB-2931] Remove access to metadata manager during compaction

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


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java:
##########
@@ -239,6 +238,60 @@ private void compactNonAlignedSeries(
     compactionWriter.endChunkGroup();
   }
 
+  private ConcurrentHashMap<String, MeasurementSchema> getMeasurementSchema(
+      String device, Set<String> measurements) throws IllegalPathException, IOException {
+    ConcurrentHashMap<String, MeasurementSchema> schemaMap = new ConcurrentHashMap<>();
+    List<TsFileResource> allResources = new LinkedList<>(seqFiles);
+    allResources.addAll(unseqFiles);
+    for (String measurement : measurements) {
+      for (TsFileResource tsFileResource : allResources) {
+        if (!tsFileResource.mayContainsDevice(device)) {
+          continue;
+        }
+        MeasurementSchema schema =
+            getMeasurementSchemaFromReader(
+                readerCacheMap.computeIfAbsent(
+                    tsFileResource,
+                    x -> {
+                      try {
+                        return new TsFileSequenceReader(x.getTsFilePath());

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