You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2022/09/05 01:41:56 UTC

[iotdb] branch rel/0.13 updated: [To rel/0.13][IOTDB-4320][ISSUE-7211] Recover insert row with null value throw NPE (#7221)

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

haonan pushed a commit to branch rel/0.13
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/0.13 by this push:
     new 60a8bec1ed [To rel/0.13][IOTDB-4320][ISSUE-7211] Recover insert row with null value throw NPE (#7221)
60a8bec1ed is described below

commit 60a8bec1ed7787c3332905f39750060a88c5f240
Author: Chen YZ <43...@users.noreply.github.com>
AuthorDate: Mon Sep 5 09:41:45 2022 +0800

    [To rel/0.13][IOTDB-4320][ISSUE-7211] Recover insert row with null value throw NPE (#7221)
---
 distribution/pom.xml                               |  2 +-
 .../db/integration/IoTDBPartialInsertionIT.java    | 13 +++++-
 .../iotdb/db/qp/physical/crud/InsertRowPlan.java   | 21 ++++++---
 .../iotdb/db/writelog/recover/LogReplayerTest.java | 51 ++++++++++++++++++++++
 testcontainer/pom.xml                              |  2 +-
 5 files changed, 81 insertions(+), 8 deletions(-)

diff --git a/distribution/pom.xml b/distribution/pom.xml
index d17809daa2..a1c52268e0 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -29,7 +29,7 @@
     </parent>
     <artifactId>iotdb-distribution</artifactId>
     <name>IoTDB Distribution</name>
-    <modules />
+    <modules/>
     <build>
         <plugins>
             <plugin>
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBPartialInsertionIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBPartialInsertionIT.java
index b8d8789edf..e2a921adff 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBPartialInsertionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBPartialInsertionIT.java
@@ -84,9 +84,10 @@ public class IoTDBPartialInsertionIT {
       statement.execute("SET STORAGE GROUP TO root.sg");
       statement.execute("CREATE TIMESERIES root.sg.d1.s1 datatype=text");
       statement.execute("CREATE TIMESERIES root.sg.d1.s2 datatype=double");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s3 datatype=text");
 
       try {
-        statement.execute("INSERT INTO root.sg.d1(time,s1,s2) VALUES(100,'test','test')");
+        statement.execute("INSERT INTO root.sg.d1(time,s1,s2,s3) VALUES(100,'test','test','test')");
       } catch (IoTDBSQLException e) {
         // ignore
       }
@@ -123,6 +124,16 @@ public class IoTDBPartialInsertionIT {
       }
       hasResultSet = statement.execute("SELECT s2 FROM root.sg.d1");
       assertTrue(hasResultSet);
+      hasResultSet = statement.execute("SELECT s3 FROM root.sg.d1");
+      assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        int cnt = 0;
+        while (resultSet.next()) {
+          cnt++;
+          assertEquals("test", resultSet.getString("root.sg.d1.s3"));
+        }
+        assertEquals(1, cnt);
+      }
       try (ResultSet resultSet = statement.getResultSet()) {
         assertFalse(resultSet.next());
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
index 85693d2879..7ebe6bd14a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
@@ -341,12 +341,13 @@ public class InsertRowPlan extends InsertPlan {
     serializeMeasurementsAndValues(stream);
   }
 
+  /** Serialize measurements and values, ignoring failed time series */
   void serializeMeasurementsAndValues(DataOutputStream stream) throws IOException {
     stream.writeInt(measurements.length - getFailedMeasurementNumber());
 
-    for (String m : measurements) {
-      if (m != null) {
-        putString(stream, m);
+    for (String measurement : measurements) {
+      if (measurement != null) {
+        putString(stream, measurement);
       }
     }
 
@@ -367,6 +368,9 @@ public class InsertRowPlan extends InsertPlan {
   private void putValues(DataOutputStream outputStream) throws QueryProcessException, IOException {
     for (int i = 0; i < values.length; i++) {
       if (values[i] == null) {
+        if (failedIndices == null || !failedIndices.contains(i)) {
+          ReadWriteIOUtils.write(TYPE_NULL, outputStream);
+        }
         continue;
       }
       // types are not determined, the situation mainly occurs when the plan uses string values
@@ -405,6 +409,9 @@ public class InsertRowPlan extends InsertPlan {
   private void putValues(ByteBuffer buffer) throws QueryProcessException {
     for (int i = 0; i < values.length; i++) {
       if (values[i] == null) {
+        if (failedIndices == null || !failedIndices.contains(i)) {
+          ReadWriteIOUtils.write(TYPE_NULL, buffer);
+        }
         continue;
       }
       // types are not determined, the situation mainly occurs when the plan uses string values
@@ -446,8 +453,11 @@ public class InsertRowPlan extends InsertPlan {
       // types are not determined, the situation mainly occurs when the plan uses string values
       // and is forwarded to other nodes
       byte typeNum = (byte) ReadWriteIOUtils.read(buffer);
-      if (typeNum == TYPE_RAW_STRING || typeNum == TYPE_NULL) {
-        values[i] = typeNum == TYPE_RAW_STRING ? ReadWriteIOUtils.readString(buffer) : null;
+      if (typeNum == TYPE_RAW_STRING) {
+        values[i] = ReadWriteIOUtils.readString(buffer);
+        continue;
+      } else if (typeNum == TYPE_NULL) {
+        values[i] = null;
         continue;
       }
       dataTypes[i] = TSDataType.values()[typeNum];
@@ -489,6 +499,7 @@ public class InsertRowPlan extends InsertPlan {
     serializeMeasurementsAndValues(buffer);
   }
 
+  /** Serialize measurements and values, ignoring failed time series */
   void serializeMeasurementsAndValues(ByteBuffer buffer) {
     buffer.putInt(measurements.length - getFailedMeasurementNumber());
 
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
index 4d31752c27..a59c46d508 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
@@ -135,6 +135,14 @@ public class LogReplayerTest {
               TSEncoding.RLE, TSEncoding.RLE, TSEncoding.RLE, TSEncoding.RLE, TSEncoding.PLAIN),
           Arrays.asList(
               compressionType, compressionType, compressionType, compressionType, compressionType));
+      for (int i = 0; i < 2; i++) {
+        IoTDB.metaManager.createTimeseries(
+            new PartialPath("root.sg.device7.s" + i),
+            TSDataType.BOOLEAN,
+            TSEncoding.PLAIN,
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
 
       // 2. delete some timeseries
       IoTDB.metaManager.deleteTimeseries(new PartialPath("root.sg.device0.sensor2"));
@@ -192,6 +200,7 @@ public class LogReplayerTest {
       }
       node.write(insertTabletPlan());
       node.write(insertAlignedTabletPlan());
+      node.write(insertRowPlanWithNullValue());
       DeletePlan deletePlan = new DeletePlan(0, 200, new PartialPath("root.sg.device0.sensor0"));
       node.write(deletePlan);
       node.close();
@@ -297,6 +306,30 @@ public class LogReplayerTest {
           }
         }
       }
+
+      // test InsertRowPlan with null value
+      for (int i = 0; i < 2; i++) {
+        MeasurementPath fullPath =
+            new MeasurementPath(
+                "root.sg.device7",
+                "s" + i,
+                new MeasurementSchema(
+                    "s" + i,
+                    TSDataType.BOOLEAN,
+                    TSEncoding.PLAIN,
+                    CompressionType.UNCOMPRESSED,
+                    Collections.emptyMap()));
+        memChunk = memTable.query(fullPath, Long.MIN_VALUE, null);
+        if (i == 1) {
+          assertNull(memChunk);
+        } else {
+          iterator = memChunk.getPointReader();
+          assertTrue(iterator.hasNextTimeValuePair());
+          TimeValuePair timeValuePair = iterator.nextTimeValuePair();
+          assertEquals(1, timeValuePair.getTimestamp());
+          assertTrue(timeValuePair.getValue().getBoolean());
+        }
+      }
     } finally {
       modFile.close();
       MultiFileLogNodeManager.getInstance()
@@ -413,4 +446,22 @@ public class LogReplayerTest {
 
     return insertTabletPlan;
   }
+
+  /**
+   * Simulate InsertRowPlan generated by SQL: insert into root.sg.device7(timestamp,s0,s1) aligned
+   * VALUES (1,True,null);
+   */
+  private InsertRowPlan insertRowPlanWithNullValue() throws IllegalPathException {
+    String deviceId = "root.sg.device7";
+
+    InsertRowPlan insertRowPlan =
+        new InsertRowPlan(
+            new PartialPath(deviceId),
+            1,
+            new String[] {"s0", "s1"},
+            new String[] {"True,", "null"});
+    insertRowPlan.setValues(new Object[] {true, null});
+    insertRowPlan.setDataTypes(new TSDataType[] {TSDataType.BOOLEAN, TSDataType.BOOLEAN});
+    return insertRowPlan;
+  }
 }
diff --git a/testcontainer/pom.xml b/testcontainer/pom.xml
index 22b03eed64..413d7eb02c 100644
--- a/testcontainer/pom.xml
+++ b/testcontainer/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <groupId>org.apache.iotdb</groupId>
         <artifactId>iotdb-parent</artifactId>
-        <version>0.13.2-SNAPSHOT</version>
+        <version>0.13.3-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>