You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ma...@apache.org on 2023/05/18 10:45:32 UTC

[iotdb] branch fix-compaction-null-pointer created (now 921e7defd5)

This is an automated email from the ASF dual-hosted git repository.

marklau99 pushed a change to branch fix-compaction-null-pointer
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 921e7defd5 [To rel/1.1][IOTDB-5897] Fix NullPointerException in compaction (#9886)

This branch includes the following new commits:

     new 921e7defd5 [To rel/1.1][IOTDB-5897] Fix NullPointerException in compaction (#9886)

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[iotdb] 01/01: [To rel/1.1][IOTDB-5897] Fix NullPointerException in compaction (#9886)

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

marklau99 pushed a commit to branch fix-compaction-null-pointer
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 921e7defd517a1d7e1a70657ec4f08860df2b2f8
Author: Liu Xuxin <37...@users.noreply.github.com>
AuthorDate: Thu May 18 18:35:10 2023 +0800

    [To rel/1.1][IOTDB-5897] Fix NullPointerException in compaction (#9886)
---
 .../execute/utils/MultiTsFileDeviceIterator.java   |  3 +-
 .../ReadChunkCompactionPerformerAlignedTest.java   | 76 ++++++++++++++++++++++
 .../compaction/utils/CompactionCheckerUtils.java   |  4 ++
 3 files changed, 82 insertions(+), 1 deletion(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/execute/utils/MultiTsFileDeviceIterator.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/execute/utils/MultiTsFileDeviceIterator.java
index 967338d1f8..6e66ac5331 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/execute/utils/MultiTsFileDeviceIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/execute/utils/MultiTsFileDeviceIterator.java
@@ -359,7 +359,8 @@ public class MultiTsFileDeviceIterator implements AutoCloseable {
       if (modification.getDevice().equals(currentDevice.left)) {
         for (int i = 0; i < valueChunkMetadataList.size(); ++i) {
           IChunkMetadata chunkMetadata = valueChunkMetadataList.get(i);
-          if (modification.getMeasurement().equals(chunkMetadata.getMeasurementUid())) {
+          if (chunkMetadata != null
+              && modification.getMeasurement().equals(chunkMetadata.getMeasurementUid())) {
             modificationForCurDevice.get(i).add(modification);
           }
         }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
index 09997f8d7b..66cba77a90 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
@@ -32,14 +32,21 @@ import org.apache.iotdb.db.engine.compaction.execute.utils.CompactionUtils;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionCheckerUtils;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
+import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
 
 import org.apache.commons.io.FileUtils;
 import org.junit.After;
@@ -652,4 +659,73 @@ public class ReadChunkCompactionPerformerAlignedTest {
             new ArrayList<>());
     CompactionCheckerUtils.validDataByValueList(originData, compactedData);
   }
+
+  @Test
+  public void testEmptyChunkWithModification() throws Exception {
+    List<IMeasurementSchema> schemas = new ArrayList<>();
+    schemas.add(new MeasurementSchema("s0", TSDataType.DOUBLE));
+    schemas.add(new MeasurementSchema("s1", TSDataType.FLOAT));
+    schemas.add(new MeasurementSchema("s2", TSDataType.INT64));
+    schemas.add(new MeasurementSchema("s3", TSDataType.INT32));
+    schemas.add(new MeasurementSchema("s4", TSDataType.TEXT));
+    schemas.add(new MeasurementSchema("s5", TSDataType.BOOLEAN));
+    Map<PartialPath, List<TimeValuePair>> originData = new HashMap<>();
+    List<TsFileResource> resources = new ArrayList<>();
+    for (int i = 1; i <= 5; i++) {
+      TsFileIOWriter writer =
+          new TsFileIOWriter(new File(dataDirectory, String.format("%d-%d-0-0.tsfile", i, i)));
+      AlignedChunkWriterImpl alignedChunkWriter = new AlignedChunkWriterImpl(schemas);
+      for (int j = i * 100; j < i * 100 + 100; j++) {
+        TsPrimitiveType[] values = {
+          new TsPrimitiveType.TsDouble(0.0D),
+          new TsPrimitiveType.TsFloat(0.0F),
+          null,
+          null,
+          new TsPrimitiveType.TsBinary(new Binary("")),
+          new TsPrimitiveType.TsBoolean(false)
+        };
+        originData
+            .computeIfAbsent(new PartialPath("root.sg.d1.s0"), k -> new ArrayList<>())
+            .add(new TimeValuePair(j, values[0]));
+        originData
+            .computeIfAbsent(new PartialPath("root.sg.d1.s1"), k -> new ArrayList<>())
+            .add(new TimeValuePair(j, values[1]));
+        originData.computeIfAbsent(new PartialPath("root.sg.d1.s2"), k -> null);
+        originData.computeIfAbsent(new PartialPath("root.sg.d1.s3"), k -> null);
+        originData
+            .computeIfAbsent(new PartialPath("root.sg.d1.s4"), k -> new ArrayList<>())
+            .add(new TimeValuePair(j, values[4]));
+        originData
+            .computeIfAbsent(new PartialPath("root.sg.d1.s5"), k -> new ArrayList<>())
+            .add(new TimeValuePair(j, values[5]));
+        alignedChunkWriter.write(j, values);
+      }
+      writer.startChunkGroup("root.sg.d1");
+      alignedChunkWriter.writeToFileWriter(writer);
+      writer.endChunkGroup();
+      writer.endFile();
+      TsFileResource resource = new TsFileResource(writer.getFile(), TsFileResourceStatus.NORMAL);
+      resource
+          .getModFile()
+          .write(new Deletion(new PartialPath("root.sg.d1.*"), i * 100, i * 100 + 20));
+      resource.getModFile().close();
+      int finalI = i;
+      originData.forEach(
+          (x, y) ->
+              y.removeIf(
+                  timeValuePair ->
+                      timeValuePair.getTimestamp() >= finalI * 100
+                          && timeValuePair.getTimestamp() < finalI * 100 + 20));
+      resources.add(resource);
+    }
+    performer.setSourceFiles(resources);
+    TsFileResource targetResource =
+        TsFileNameGenerator.getInnerCompactionTargetFileResource(resources, true);
+    performer.setTargetFiles(Collections.singletonList(targetResource));
+    performer.setSummary(new CompactionTaskSummary());
+    performer.perform();
+    Assert.assertTrue(targetResource.getTsFile().exists());
+    RestorableTsFileIOWriter checkWriter = new RestorableTsFileIOWriter(targetResource.getTsFile());
+    Assert.assertFalse(checkWriter.hasCrashed());
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionCheckerUtils.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionCheckerUtils.java
index 6ba310d439..68320ee32f 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionCheckerUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionCheckerUtils.java
@@ -545,6 +545,10 @@ public class CompactionCheckerUtils {
       List<TimeValuePair> expectedTimeValueList = expectedData.get(path);
       List<TimeValuePair> actualTimeValueList = actualData.get(path);
 
+      if (actualTimeValueList == null) {
+        assertNull(expectedTimeValueList);
+        continue;
+      }
       assertEquals(expectedTimeValueList.size(), actualTimeValueList.size());
 
       for (int i = 0; i < expectedTimeValueList.size(); ++i) {