You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by xi...@apache.org on 2023/04/20 07:20:20 UTC

[iotdb] branch master updated: Add a cache to avoid initialize duplicated device id object in write process (#9644)

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

xingtanzjr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 348763f208 Add a cache to avoid initialize duplicated device id object in write process (#9644)
348763f208 is described below

commit 348763f20879c85b39c0ebde1bfccc7134f5b2e8
Author: Haonan <hh...@outlook.com>
AuthorDate: Thu Apr 20 15:20:12 2023 +0800

    Add a cache to avoid initialize duplicated device id object in write process (#9644)
---
 docs/UserGuide/Reference/Common-Config-Manual.md   | 11 +++-
 .../zh/UserGuide/Reference/Common-Config-Manual.md |  9 +++
 .../resources/conf/iotdb-common.properties         |  4 ++
 .../org/apache/iotdb/db/audit/AuditLogger.java     |  8 ++-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 10 ++++
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  6 ++
 .../db/metadata/cache/DataNodeDevicePathCache.java | 69 ++++++++++++++++++++++
 .../db/mpp/plan/parser/StatementGenerator.java     | 25 +++++---
 .../iotdb/db/protocol/mqtt/MPPPublishHandler.java  |  5 +-
 .../v1/handler/StatementConstructionHandler.java   |  5 +-
 .../v2/handler/StatementConstructionHandler.java   |  5 +-
 .../impl/DataNodeInternalRPCServiceImpl.java       |  2 +
 .../cache/DataNodeDevicePathCacheTest.java         | 62 +++++++++++++++++++
 13 files changed, 203 insertions(+), 18 deletions(-)

diff --git a/docs/UserGuide/Reference/Common-Config-Manual.md b/docs/UserGuide/Reference/Common-Config-Manual.md
index 424e6d9295..562486bc03 100644
--- a/docs/UserGuide/Reference/Common-Config-Manual.md
+++ b/docs/UserGuide/Reference/Common-Config-Manual.md
@@ -797,7 +797,16 @@ Different configuration parameters take effect in the following three ways:
 |Description| When there exists old version(v2) TsFile, how many thread will be set up to perform upgrade tasks |
 |   Type    | Int32                                                                                             |
 |  Default  | 1                                                                                                 |
-| Effective | After restarting system                                                                           |                                                                        |
+| Effective | After restarting system                                                                           |
+
+* device\_path\_cache\_size
+
+|   Name    | device\_path\_cache\_size                                                                                                 |
+|:---------:|:--------------------------------------------------------------------------------------------------------------------------|
+|Description| The max size of the device path cache. This cache is for avoiding initialize duplicated device id object in write process |
+|   Type    | Int32                                                                                                                     |
+|  Default  | 500000                                                                                                                    |
+| Effective | After restarting system                                                                                                   |
 
 * insert\_multi\_tablet\_enable\_multithreading\_column\_threshold
 
diff --git a/docs/zh/UserGuide/Reference/Common-Config-Manual.md b/docs/zh/UserGuide/Reference/Common-Config-Manual.md
index c8e48ed438..a1a22cd8f0 100644
--- a/docs/zh/UserGuide/Reference/Common-Config-Manual.md
+++ b/docs/zh/UserGuide/Reference/Common-Config-Manual.md
@@ -847,6 +847,15 @@ IoTDB ConfigNode 和 DataNode 的公共配置参数位于 `conf` 目录下。
 |    默认值    | 1                               |
 | 改后生效方式 | 重启服务生效                          |
 
+* device\_path\_cache\_size
+
+|     名字     | device\_path\_cache\_size                             |
+| :----------: |:------------------------------------------------------|
+|     描述     | Device Path 缓存的最大数量,这个缓存可以避免写入过程中重复的 Device Path 对象创建 |
+|     类型     | Int32                                                 |
+|    默认值    | 500000                                                |
+| 改后生效方式 | 重启服务生效                                                |
+
 * insert\_multi\_tablet\_enable\_multithreading\_column\_threshold
 
 |     名字     | insert\_multi\_tablet\_enable\_multithreading\_column\_threshold |
diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index 038b1f894d..61b0bbc4a7 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -528,6 +528,10 @@ cluster_name=defaultCluster
 # Datatype: int
 # upgrade_thread_count=1
 
+# The max size of the device path cache. This cache is for avoiding initialize duplicated device id object in write process.
+# Datatype: int
+# device_path_cache_size=500000
+
 ####################
 ### Compaction Configurations
 ####################
diff --git a/server/src/main/java/org/apache/iotdb/db/audit/AuditLogger.java b/server/src/main/java/org/apache/iotdb/db/audit/AuditLogger.java
index 0355fbbfcd..14acd8c59b 100644
--- a/server/src/main/java/org/apache/iotdb/db/audit/AuditLogger.java
+++ b/server/src/main/java/org/apache/iotdb/db/audit/AuditLogger.java
@@ -20,10 +20,10 @@ package org.apache.iotdb.db.audit;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.mpp.plan.Coordinator;
 import org.apache.iotdb.db.mpp.plan.analyze.ClusterPartitionFetcher;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
@@ -63,6 +63,9 @@ public class AuditLogger {
 
   private static final SessionManager SESSION_MANAGER = SessionManager.getInstance();
 
+  private static final DataNodeDevicePathCache DEVICE_PATH_CACHE =
+      DataNodeDevicePathCache.getInstance();
+
   private AuditLogger() {
     // empty constructor
   }
@@ -72,7 +75,8 @@ public class AuditLogger {
       String log, String address, String username)
       throws IoTDBConnectionException, IllegalPathException, QueryProcessException {
     InsertRowStatement insertStatement = new InsertRowStatement();
-    insertStatement.setDevicePath(new PartialPath(String.format(AUDIT_LOG_DEVICE, username)));
+    insertStatement.setDevicePath(
+        DEVICE_PATH_CACHE.getPartialPath(String.format(AUDIT_LOG_DEVICE, username)));
     insertStatement.setTime(DateTimeUtils.currentTime());
     insertStatement.setMeasurements(new String[] {LOG, USERNAME, ADDRESS});
     insertStatement.setAligned(false);
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 97dd9ccdf7..982447e91b 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -960,6 +960,8 @@ public class IoTDBConfig {
    */
   private int partitionCacheSize = 1000;
 
+  private int devicePathCacheSize = 500_000;
+
   /** Cache size of user and role */
   private int authorCacheSize = 100;
 
@@ -3168,6 +3170,14 @@ public class IoTDBConfig {
     this.partitionCacheSize = partitionCacheSize;
   }
 
+  public int getDevicePathCacheSize() {
+    return devicePathCacheSize;
+  }
+
+  public void setDevicePathCacheSize(int devicePathCacheSize) {
+    this.devicePathCacheSize = devicePathCacheSize;
+  }
+
   public int getAuthorCacheSize() {
     return authorCacheSize;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 5581bacde9..c8548835db 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -774,6 +774,12 @@ public class IoTDBDescriptor {
     conf.setKerberosPrincipal(
         properties.getProperty("kerberos_principal", conf.getKerberosPrincipal()));
 
+    // the size of device path cache
+    conf.setDevicePathCacheSize(
+        Integer.parseInt(
+            properties.getProperty(
+                "device_path_cache_size", String.valueOf(conf.getDevicePathCacheSize()))));
+
     // the num of memtables in each database
     conf.setConcurrentWritingTimePartition(
         Integer.parseInt(
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCache.java b/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCache.java
new file mode 100644
index 0000000000..0695bab048
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCache.java
@@ -0,0 +1,69 @@
+/*
+ * 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.metadata.cache;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/** This cache is for reducing duplicated DeviceId PartialPath initialization in write process. */
+public class DataNodeDevicePathCache {
+
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private final Cache<String, PartialPath> devicePathCache;
+
+  private DataNodeDevicePathCache() {
+    devicePathCache = Caffeine.newBuilder().maximumSize(config.getDevicePathCacheSize()).build();
+  }
+
+  public static DataNodeDevicePathCache getInstance() {
+    return DataNodeDevicePathCache.DataNodeDevicePathCacheHolder.INSTANCE;
+  }
+
+  /** singleton pattern. */
+  private static class DataNodeDevicePathCacheHolder {
+    private static final DataNodeDevicePathCache INSTANCE = new DataNodeDevicePathCache();
+  }
+
+  public PartialPath getPartialPath(String deviceId) throws IllegalPathException {
+    try {
+      return devicePathCache.get(
+          deviceId,
+          path -> {
+            try {
+              return new PartialPath(path);
+            } catch (IllegalPathException e) {
+              throw new IllegalArgumentException(e);
+            }
+          });
+    } catch (IllegalArgumentException e) {
+      throw new IllegalPathException(deviceId);
+    }
+  }
+
+  public void cleanUp() {
+    devicePathCache.cleanUp();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
index 432b76a718..f29b8016e5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.db.constant.SqlConstant;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.metadata.template.TemplateQueryType;
 import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
 import org.apache.iotdb.db.mpp.plan.expression.binary.GreaterEqualExpression;
@@ -120,6 +121,9 @@ public class StatementGenerator {
   private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
       PerformanceOverviewMetrics.getInstance();
 
+  private static final DataNodeDevicePathCache DEVICE_PATH_CACHE =
+      DataNodeDevicePathCache.getInstance();
+
   public static Statement createStatement(String sql, ZoneId zoneId) {
     return invokeParser(sql, zoneId);
   }
@@ -256,7 +260,8 @@ public class StatementGenerator {
     final long startTime = System.nanoTime();
     // construct insert statement
     InsertRowStatement insertStatement = new InsertRowStatement();
-    insertStatement.setDevicePath(new PartialPath(insertRecordReq.getPrefixPath()));
+    insertStatement.setDevicePath(
+        DEVICE_PATH_CACHE.getPartialPath(insertRecordReq.getPrefixPath()));
     insertStatement.setTime(insertRecordReq.getTimestamp());
     insertStatement.setMeasurements(insertRecordReq.getMeasurements().toArray(new String[0]));
     insertStatement.setAligned(insertRecordReq.isAligned);
@@ -270,7 +275,8 @@ public class StatementGenerator {
     final long startTime = System.nanoTime();
     // construct insert statement
     InsertRowStatement insertStatement = new InsertRowStatement();
-    insertStatement.setDevicePath(new PartialPath(insertRecordReq.getPrefixPath()));
+    insertStatement.setDevicePath(
+        DEVICE_PATH_CACHE.getPartialPath(insertRecordReq.getPrefixPath()));
     insertStatement.setTime(insertRecordReq.getTimestamp());
     insertStatement.setMeasurements(insertRecordReq.getMeasurements().toArray(new String[0]));
     insertStatement.setDataTypes(new TSDataType[insertStatement.getMeasurements().length]);
@@ -286,7 +292,8 @@ public class StatementGenerator {
     final long startTime = System.nanoTime();
     // construct insert statement
     InsertTabletStatement insertStatement = new InsertTabletStatement();
-    insertStatement.setDevicePath(new PartialPath(insertTabletReq.getPrefixPath()));
+    insertStatement.setDevicePath(
+        DEVICE_PATH_CACHE.getPartialPath(insertTabletReq.getPrefixPath()));
     insertStatement.setMeasurements(insertTabletReq.getMeasurements().toArray(new String[0]));
     insertStatement.setTimes(
         QueryDataSetUtils.readTimesFromBuffer(insertTabletReq.timestamps, insertTabletReq.size));
@@ -318,7 +325,7 @@ public class StatementGenerator {
     List<InsertTabletStatement> insertTabletStatementList = new ArrayList<>();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
       InsertTabletStatement insertTabletStatement = new InsertTabletStatement();
-      insertTabletStatement.setDevicePath(new PartialPath(req.prefixPaths.get(i)));
+      insertTabletStatement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(req.prefixPaths.get(i)));
       insertTabletStatement.setMeasurements(req.measurementsList.get(i).toArray(new String[0]));
       insertTabletStatement.setTimes(
           QueryDataSetUtils.readTimesFromBuffer(req.timestampsList.get(i), req.sizeList.get(i)));
@@ -357,7 +364,7 @@ public class StatementGenerator {
     List<InsertRowStatement> insertRowStatementList = new ArrayList<>();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
       InsertRowStatement statement = new InsertRowStatement();
-      statement.setDevicePath(new PartialPath(req.getPrefixPaths().get(i)));
+      statement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(req.getPrefixPaths().get(i)));
       statement.setMeasurements(req.getMeasurementsList().get(i).toArray(new String[0]));
       statement.setTime(req.getTimestamps().get(i));
       statement.fillValues(req.valuesList.get(i));
@@ -381,7 +388,7 @@ public class StatementGenerator {
     List<InsertRowStatement> insertRowStatementList = new ArrayList<>();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
       InsertRowStatement statement = new InsertRowStatement();
-      statement.setDevicePath(new PartialPath(req.getPrefixPaths().get(i)));
+      statement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(req.getPrefixPaths().get(i)));
       addMeasurementAndValue(
           statement, req.getMeasurementsList().get(i), req.getValuesList().get(i));
       statement.setDataTypes(new TSDataType[statement.getMeasurements().length]);
@@ -404,7 +411,7 @@ public class StatementGenerator {
     final long startTime = System.nanoTime();
     // construct insert statement
     InsertRowsOfOneDeviceStatement insertStatement = new InsertRowsOfOneDeviceStatement();
-    insertStatement.setDevicePath(new PartialPath(req.prefixPath));
+    insertStatement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(req.prefixPath));
     List<InsertRowStatement> insertRowStatementList = new ArrayList<>();
     for (int i = 0; i < req.timestamps.size(); i++) {
       InsertRowStatement statement = new InsertRowStatement();
@@ -429,7 +436,7 @@ public class StatementGenerator {
     final long startTime = System.nanoTime();
     // construct insert statement
     InsertRowsOfOneDeviceStatement insertStatement = new InsertRowsOfOneDeviceStatement();
-    insertStatement.setDevicePath(new PartialPath(req.prefixPath));
+    insertStatement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(req.prefixPath));
     List<InsertRowStatement> insertRowStatementList = new ArrayList<>();
     for (int i = 0; i < req.timestamps.size(); i++) {
       InsertRowStatement statement = new InsertRowStatement();
@@ -820,7 +827,7 @@ public class StatementGenerator {
             + TsFileConstant.PATH_SEPARATOR
             + recordModelMetricsReq.getTrialId();
     InsertRowStatement insertRowStatement = new InsertRowStatement();
-    insertRowStatement.setDevicePath(new PartialPath(path));
+    insertRowStatement.setDevicePath(DEVICE_PATH_CACHE.getPartialPath(path));
     insertRowStatement.setTime(recordModelMetricsReq.getTimestamp());
     insertRowStatement.setMeasurements(recordModelMetricsReq.getMetrics().toArray(new String[0]));
     insertRowStatement.setAligned(true);
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
index 7e8cae62b1..90cb362c32 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
@@ -19,10 +19,10 @@ package org.apache.iotdb.db.protocol.mqtt;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.conf.IoTDBConstant.ClientVersion;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.auth.AuthorityChecker;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.mpp.plan.Coordinator;
 import org.apache.iotdb.db.mpp.plan.analyze.ClusterPartitionFetcher;
 import org.apache.iotdb.db.mpp.plan.analyze.IPartitionFetcher;
@@ -136,7 +136,8 @@ public class MPPPublishHandler extends AbstractInterceptHandler {
       TSStatus tsStatus = null;
       try {
         InsertRowStatement statement = new InsertRowStatement();
-        statement.setDevicePath(new PartialPath(event.getDevice()));
+        statement.setDevicePath(
+            DataNodeDevicePathCache.getInstance().getPartialPath(event.getDevice()));
         statement.setTime(event.getTimestamp());
         statement.setMeasurements(event.getMeasurements().toArray(new String[0]));
         if (event.getDataTypes() == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/handler/StatementConstructionHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/handler/StatementConstructionHandler.java
index c0d422601d..d53795dc7d 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/handler/StatementConstructionHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/handler/StatementConstructionHandler.java
@@ -18,8 +18,8 @@
 package org.apache.iotdb.db.protocol.rest.v1.handler;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.WriteProcessRejectException;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
 import org.apache.iotdb.db.protocol.rest.v1.model.InsertTabletRequest;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -38,7 +38,8 @@ public class StatementConstructionHandler {
       throws IllegalPathException, WriteProcessRejectException {
     // construct insert statement
     InsertTabletStatement insertStatement = new InsertTabletStatement();
-    insertStatement.setDevicePath(new PartialPath(insertTabletRequest.getDeviceId()));
+    insertStatement.setDevicePath(
+        DataNodeDevicePathCache.getInstance().getPartialPath(insertTabletRequest.getDeviceId()));
     insertStatement.setMeasurements(insertTabletRequest.getMeasurements().toArray(new String[0]));
     List<List<Object>> rawData = insertTabletRequest.getValues();
     List<String> rawDataType = insertTabletRequest.getDataTypes();
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/handler/StatementConstructionHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/handler/StatementConstructionHandler.java
index 4d643862d9..870c2c7305 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/handler/StatementConstructionHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/handler/StatementConstructionHandler.java
@@ -18,8 +18,8 @@
 package org.apache.iotdb.db.protocol.rest.v2.handler;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.WriteProcessRejectException;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
 import org.apache.iotdb.db.protocol.rest.v2.model.InsertTabletRequest;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -38,7 +38,8 @@ public class StatementConstructionHandler {
       throws IllegalPathException, WriteProcessRejectException {
     // construct insert statement
     InsertTabletStatement insertStatement = new InsertTabletStatement();
-    insertStatement.setDevicePath(new PartialPath(insertTabletRequest.getDevice()));
+    insertStatement.setDevicePath(
+        DataNodeDevicePathCache.getInstance().getPartialPath(insertTabletRequest.getDevice()));
     insertStatement.setMeasurements(insertTabletRequest.getMeasurements().toArray(new String[0]));
     List<List<Object>> rawData = insertTabletRequest.getValues();
     List<String> rawDataType = insertTabletRequest.getDataTypes();
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
index 4704bf122a..7a9e8326bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -60,6 +60,7 @@ import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.settle.SettleRequestHandler;
 import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.metadata.cache.DataNodeDevicePathCache;
 import org.apache.iotdb.db.metadata.cache.DataNodeSchemaCache;
 import org.apache.iotdb.db.metadata.query.info.ITimeSeriesSchemaInfo;
 import org.apache.iotdb.db.metadata.query.reader.ISchemaReader;
@@ -1617,6 +1618,7 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface
     // TODO what need to clean?
     ClusterPartitionFetcher.getInstance().invalidAllCache();
     DataNodeSchemaCache.getInstance().cleanUp();
+    DataNodeDevicePathCache.getInstance().cleanUp();
     return status;
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCacheTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCacheTest.java
new file mode 100644
index 0000000000..ad86fa3602
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeDevicePathCacheTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.metadata.cache;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class DataNodeDevicePathCacheTest {
+
+  DataNodeDevicePathCache dataNodeDevicePathCache;
+
+  @Before
+  public void setUp() throws Exception {
+    dataNodeDevicePathCache = DataNodeDevicePathCache.getInstance();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    dataNodeDevicePathCache.cleanUp();
+  }
+
+  @Test
+  public void testGetPartialPath() {
+    try {
+      dataNodeDevicePathCache.getPartialPath("root.sg.d1");
+    } catch (IllegalPathException e) {
+      Assert.fail();
+    }
+  }
+
+  @Test(expected = IllegalPathException.class)
+  public void testGetIllegalPartialPath() throws Exception {
+    try {
+      dataNodeDevicePathCache.getPartialPath("root.sg.1");
+    } catch (IllegalPathException e) {
+      Assert.assertEquals("root.sg.1 is not a legal path", e.getMessage());
+      throw e;
+    }
+    Assert.fail("root.sg.1 should be an illegal path");
+  }
+}