You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/04/23 08:31:33 UTC

[iotdb] branch master updated: [IOTDB-2930]Fix concurrent UnPin bug & Improve template implementation (#5647)

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

qiaojialin 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 9685c5b67b [IOTDB-2930]Fix concurrent UnPin bug & Improve template implementation (#5647)
9685c5b67b is described below

commit 9685c5b67b06bf30d80454ccce424fdf22f44ce9
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Sat Apr 23 16:31:27 2022 +0800

    [IOTDB-2930]Fix concurrent UnPin bug & Improve template implementation (#5647)
---
 .../iotdb/db/metadata/mtree/IMTreeBelowSG.java     |   3 +-
 .../db/metadata/mtree/MTreeBelowSGCachedImpl.java  |  90 +++++++++++------
 .../db/metadata/mtree/MTreeBelowSGMemoryImpl.java  |  87 +++++++++++------
 .../mtree/store/disk/cache/CacheManager.java       |  35 ++++---
 .../schemaregion/SchemaRegionMemoryImpl.java       | 103 ++++++++++----------
 .../schemaregion/SchemaRegionSchemaFileImpl.java   | 106 +++++++++++----------
 6 files changed, 247 insertions(+), 177 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java
index f556b2530c..d888e7f696 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java
@@ -292,7 +292,8 @@ public interface IMTreeBelowSG {
    * @return index on full path of the node which matches all measurements path with its
    *     upperTemplate.
    */
-  int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath) throws MetadataException;
+  int getMountedNodeIndexOnMeasurementPath(PartialPath devicePath, String[] measurements)
+      throws MetadataException;
 
   List<String> getPathsSetOnTemplate(String templateName) throws MetadataException;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
index 962b5f1aed..10ec6e3d3d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
@@ -80,6 +80,7 @@ import java.util.stream.Stream;
 
 import static java.util.stream.Collectors.toList;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
 import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
 
@@ -111,7 +112,7 @@ import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTim
 public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
 
   private CachedMTreeStore store;
-  private IStorageGroupMNode storageGroupMNode;
+  private volatile IStorageGroupMNode storageGroupMNode;
   private int levelOfSG;
 
   // region MTree initialization, clear and serialization
@@ -1289,53 +1290,80 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    * @return index on full path of the node which matches all measurements path with its
    *     upperTemplate.
    */
-  @Override
-  public int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath)
+  public int getMountedNodeIndexOnMeasurementPath(PartialPath devicePath, String[] measurements)
       throws MetadataException {
-    String[] fullPathNodes = measurementPath.getNodes();
+    String[] nodes = devicePath.getNodes();
     IMNode cur = storageGroupMNode;
     IMNode child;
     Template upperTemplate = cur.getSchemaTemplate();
+    int index = levelOfSG + 1;
+    boolean attemptToUseTemplate = false;
+
     try {
-      for (int index = levelOfSG + 1; index < fullPathNodes.length; index++) {
+      // If there are nodes of target path on MTree, use it as possible.
+      for (; index < nodes.length; index++) {
         upperTemplate = cur.getSchemaTemplate() != null ? cur.getSchemaTemplate() : upperTemplate;
-        child = store.getChild(cur, fullPathNodes[index]);
+        child = store.getChild(cur, nodes[index]);
         if (child == null) {
-          if (upperTemplate != null) {
-            // for this fullPath, cur is the last node on MTree
-            // since upperTemplate exists, need to find the matched suffix path of fullPath and
-            // template
-            String suffixPath =
-                new PartialPath(Arrays.copyOfRange(fullPathNodes, index, fullPathNodes.length))
-                    .toString();
-
-            // if suffix matches template, then fullPathNodes[index-1] should be the node to use
-            // template on MTree
-            if (upperTemplate.hasSchema(suffixPath)) {
-              return index - 1;
-            }
-
-            // if suffix doesn't match, but first node name matched, it's an overlap with template
-            // cast exception for now
-            if (upperTemplate.getDirectNode(fullPathNodes[index]) != null) {
-              throw new TemplateImcompatibeException(
-                  measurementPath.getFullPath(), upperTemplate.getName(), fullPathNodes[index]);
-            }
-          } else {
+          if (upperTemplate == null) {
             // no matched child, no template, need to create device node as logical device path
-            return fullPathNodes.length - 1;
+            return nodes.length;
+          } else {
+            attemptToUseTemplate = true;
+            break;
           }
         } else {
           // has child on MTree
           cur = child;
         }
       }
+    } finally {
+      if (index > levelOfSG + 1) {
+        unPinPath(cur);
+      }
+    }
 
+    if (!attemptToUseTemplate) {
       // all nodes on path exist in MTree, device node should be the penultimate one
-      return fullPathNodes.length - 1;
-    } finally {
-      unPinPath(cur);
+      return nodes.length;
     }
+
+    // The resting part of target path not exists on MTree, thus try to use template.
+    for (; index < nodes.length; index++) {
+      int fullPathLength = nodes.length - index + 1;
+      String[] suffixNodes = new String[fullPathLength];
+      System.arraycopy(nodes, index, suffixNodes, 0, nodes.length - index);
+      boolean hasAllMeasurements = true;
+
+      for (String measurement : measurements) {
+        // for this fullPath, cur is the last node on MTree
+        // since upperTemplate exists, need to find the matched suffix path of fullPath and
+        // template
+        suffixNodes[fullPathLength - 1] = measurement;
+        String suffixPath = String.join(String.valueOf(PATH_SEPARATOR), suffixNodes);
+
+        if (upperTemplate.hasSchema(suffixPath)) {
+          continue;
+        }
+
+        // if suffix doesn't match, but first node name matched, it's an overlap with template
+        // cast exception for now
+        if (upperTemplate.getDirectNode(nodes[index]) != null) {
+          throw new TemplateImcompatibeException(
+              devicePath.concatNode(measurement).getFullPath(),
+              upperTemplate.getName(),
+              nodes[index]);
+        }
+
+        hasAllMeasurements = false;
+      }
+
+      if (hasAllMeasurements) {
+        return index - 1;
+      }
+    }
+
+    return nodes.length;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
index 0b53297b3d..1bba466416 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
@@ -79,6 +79,7 @@ import java.util.stream.Stream;
 
 import static java.util.stream.Collectors.toList;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
 import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
 
@@ -111,7 +112,7 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
 
   // this implementation is based on memory, thus only MTree write operation must invoke MTreeStore
   private MemMTreeStore store;
-  private IStorageGroupMNode storageGroupMNode;
+  private volatile IStorageGroupMNode storageGroupMNode;
   private int levelOfSG;
 
   // region MTree initialization, clear and serialization
@@ -1159,44 +1160,29 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
    * created yet. The result is used for getDeviceNodeWithAutoCreate, which return corresponding
    * IMNode on MTree.
    *
-   * @return index on full path of the node which matches all measurements path with its
+   * @return index on full path of the node which matches all measurements' path with its
    *     upperTemplate.
    */
-  @Override
-  public int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath)
+  public int getMountedNodeIndexOnMeasurementPath(PartialPath devicePath, String[] measurements)
       throws MetadataException {
-    String[] fullPathNodes = measurementPath.getNodes();
+    String[] nodes = devicePath.getNodes();
     IMNode cur = storageGroupMNode;
     IMNode child;
     Template upperTemplate = cur.getSchemaTemplate();
+    int index;
+    boolean attemptToUseTemplate = false;
 
-    for (int index = levelOfSG + 1; index < fullPathNodes.length; index++) {
+    // If there are nodes of target path on MTree, use it as possible.
+    for (index = levelOfSG + 1; index < nodes.length; index++) {
       upperTemplate = cur.getSchemaTemplate() != null ? cur.getSchemaTemplate() : upperTemplate;
-      child = cur.getChild(fullPathNodes[index]);
+      child = cur.getChild(nodes[index]);
       if (child == null) {
-        if (upperTemplate != null) {
-          // for this fullPath, cur is the last node on MTree
-          // since upperTemplate exists, need to find the matched suffix path of fullPath and
-          // template
-          String suffixPath =
-              new PartialPath(Arrays.copyOfRange(fullPathNodes, index, fullPathNodes.length))
-                  .toString();
-
-          // if suffix matches template, then fullPathNodes[index-1] should be the node to use
-          // template on MTree
-          if (upperTemplate.hasSchema(suffixPath)) {
-            return index - 1;
-          }
-
-          // if suffix doesn't match, but first node name matched, it's an overlap with template
-          // cast exception for now
-          if (upperTemplate.getDirectNode(fullPathNodes[index]) != null) {
-            throw new TemplateImcompatibeException(
-                measurementPath.getFullPath(), upperTemplate.getName(), fullPathNodes[index]);
-          }
-        } else {
+        if (upperTemplate == null) {
           // no matched child, no template, need to create device node as logical device path
-          return fullPathNodes.length - 1;
+          return nodes.length;
+        } else {
+          attemptToUseTemplate = true;
+          break;
         }
       } else {
         // has child on MTree
@@ -1204,8 +1190,47 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
       }
     }
 
-    // all nodes on path exist in MTree, device node should be the penultimate one
-    return fullPathNodes.length - 1;
+    if (!attemptToUseTemplate) {
+      // all nodes on path exist in MTree, device node should be the penultimate one
+      return nodes.length;
+    }
+
+    // The resting part of target path not exists on MTree, thus try to use template.
+    for (; index < nodes.length; index++) {
+      int fullPathLength = nodes.length - index + 1;
+      String[] suffixNodes = new String[fullPathLength];
+      System.arraycopy(nodes, index, suffixNodes, 0, nodes.length - index);
+      boolean hasAllMeasurements = true;
+
+      for (String measurement : measurements) {
+        // for this fullPath, cur is the last node on MTree
+        // since upperTemplate exists, need to find the matched suffix path of fullPath and
+        // template
+        suffixNodes[fullPathLength - 1] = measurement;
+        String suffixPath = String.join(String.valueOf(PATH_SEPARATOR), suffixNodes);
+
+        if (upperTemplate.hasSchema(suffixPath)) {
+          continue;
+        }
+
+        // if suffix doesn't match, but first node name matched, it's an overlap with template
+        // cast exception for now
+        if (upperTemplate.getDirectNode(nodes[index]) != null) {
+          throw new TemplateImcompatibeException(
+              devicePath.concatNode(measurement).getFullPath(),
+              upperTemplate.getName(),
+              nodes[index]);
+        }
+
+        hasAllMeasurements = false;
+      }
+
+      if (hasAllMeasurements) {
+        return index - 1;
+      }
+    }
+
+    return nodes.length;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/cache/CacheManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/cache/CacheManager.java
index 7ff0ba78c5..39fb5414bc 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/cache/CacheManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/cache/CacheManager.java
@@ -427,24 +427,29 @@ public abstract class CacheManager implements ICacheManager {
     if (cacheEntry == null) {
       return false;
     }
-    cacheEntry.unPin();
-    if (!cacheEntry.isPinned()) {
-      memManager.releasePinnedMemResource(node);
-      IMNode parent = node.getParent();
-      while (!node.isStorageGroup()) {
-        node = parent;
-        parent = node.getParent();
-        cacheEntry = getCacheEntry(node);
-        cacheEntry.unPin();
-        if (cacheEntry.isPinned()) {
-          break;
-        }
+
+    return doUnPin(node);
+  }
+
+  private boolean doUnPin(IMNode node) {
+    CacheEntry cacheEntry = getCacheEntry(node);
+
+    boolean isPinStatusChanged = false;
+    synchronized (cacheEntry) {
+      cacheEntry.unPin();
+      if (!cacheEntry.isPinned()) {
+        isPinStatusChanged = true;
         memManager.releasePinnedMemResource(node);
       }
-      return true;
-    } else {
-      return false;
     }
+
+    if (isPinStatusChanged) {
+      if (!node.isStorageGroup()) {
+        doUnPin(node.getParent());
+      }
+    }
+
+    return isPinStatusChanged;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
index 0717d881f3..e3a6e3268d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
@@ -718,14 +718,14 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
    *
    * @param path path
    */
-  public IMNode getDeviceNodeWithAutoCreate(PartialPath path, boolean autoCreateSchema)
+  private IMNode getDeviceNodeWithAutoCreate(PartialPath path)
       throws IOException, MetadataException {
     IMNode node;
     try {
       return mNodeCache.get(path);
     } catch (Exception e) {
       if (e.getCause() instanceof MetadataException) {
-        if (!autoCreateSchema) {
+        if (!config.isAutoCreateSchemaEnabled()) {
           throw new PathNotExistException(path.getFullPath());
         }
       } else {
@@ -740,13 +740,8 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
     return node;
   }
 
-  public IMNode getDeviceNodeWithAutoCreate(PartialPath path)
-      throws MetadataException, IOException {
-    return getDeviceNodeWithAutoCreate(path, config.isAutoCreateSchemaEnabled());
-  }
-
   public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws MetadataException {
-    IMNode node = mtree.getDeviceNodeWithAutoCreating(plan.getPath());
+    mtree.getDeviceNodeWithAutoCreating(plan.getPath());
     if (!isRecovering) {
       try {
         logWriter.autoCreateDeviceMNode(plan);
@@ -1134,8 +1129,7 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   }
 
   public IMeasurementMNode getMeasurementMNode(PartialPath fullPath) throws MetadataException {
-    IMeasurementMNode measurementMNode = mtree.getMeasurementMNode(fullPath);
-    return measurementMNode;
+    return mtree.getMeasurementMNode(fullPath);
   }
 
   /**
@@ -1370,37 +1364,12 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
     PartialPath devicePath = plan.getDevicePath();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-    IMNode deviceMNode = null;
+    IMNode deviceMNode;
 
     // 1. get device node, set using template if accessed.
-    boolean mountedNodeFound = false;
-    boolean isDeviceInTemplate = false;
-    // check every measurement path
-    for (String measurementId : measurementList) {
-      PartialPath fullPath = devicePath.concatNode(measurementId);
-      int index = mtree.getMountedNodeIndexOnMeasurementPath(fullPath);
-      if ((index != fullPath.getNodeLength() - 1) && !mountedNodeFound) {
-        // this measurement is in template, need to assure mounted node exists and set using
-        // template.
-        // Without allowing overlap of template and MTree, this block run only once
-        String[] mountedPathNodes = Arrays.copyOfRange(fullPath.getNodes(), 0, index + 1);
-        IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
-        if (!mountedNode.isUseTemplate()) {
-          mountedNode = setUsingSchemaTemplate(mountedNode);
-        }
-        mountedNodeFound = true;
-        if (index < devicePath.getNodeLength() - 1) {
-          deviceMNode =
-              mountedNode
-                  .getUpperTemplate()
-                  .getPathNodeInTemplate(
-                      new PartialPath(
-                          Arrays.copyOfRange(
-                              devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
-          isDeviceInTemplate = true;
-        }
-      }
-    }
+    deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
+    boolean isDeviceInTemplate = deviceMNode != null;
+
     // get logical device node, may be in template. will be multiple if overlap is allowed.
     if (!isDeviceInTemplate) {
       deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
@@ -1479,6 +1448,38 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
     return deviceMNode;
   }
 
+  private IMNode getDeviceInTemplateIfUsingTemplate(
+      PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
+    // 1. get device node, set using template if accessed.
+    IMNode deviceMNode = null;
+
+    // check every measurement path
+    int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
+    if (index == devicePath.getNodeLength()) {
+      return null;
+    }
+
+    // this measurement is in template, need to assure mounted node exists and set using
+    // template.
+    // Without allowing overlap of template and MTree, this block run only once
+    String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
+    IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
+    if (!mountedNode.isUseTemplate()) {
+      mountedNode = setUsingSchemaTemplate(mountedNode);
+    }
+    if (index < devicePath.getNodeLength() - 1) {
+      deviceMNode =
+          mountedNode
+              .getUpperTemplate()
+              .getPathNodeInTemplate(
+                  new PartialPath(
+                      Arrays.copyOfRange(
+                          devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
+    }
+
+    return deviceMNode;
+  }
+
   private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
       InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
       throws MetadataException {
@@ -1524,19 +1525,23 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
       throws MetadataException {
     Template curTemplate = deviceMNode.getUpperTemplate();
-    if (curTemplate != null) {
-      IMeasurementSchema schema = curTemplate.getSchema(measurement);
-      if (!deviceMNode.isUseTemplate()) {
-        deviceMNode = setUsingSchemaTemplate(deviceMNode);
-      }
 
-      if (schema != null) {
-        return MeasurementMNode.getMeasurementMNode(
-            deviceMNode.getAsEntityMNode(), measurement, schema, null);
-      }
+    if (curTemplate == null) {
+      return null;
+    }
+
+    IMeasurementSchema schema = curTemplate.getSchema(measurement);
+
+    if (schema == null) {
       return null;
     }
-    return null;
+
+    if (!deviceMNode.isUseTemplate()) {
+      deviceMNode = setUsingSchemaTemplate(deviceMNode);
+    }
+
+    return MeasurementMNode.getMeasurementMNode(
+        deviceMNode.getAsEntityMNode(), measurement, schema, null);
   }
 
   /** create timeseries ignoring PathAlreadyExistException */
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
index fd8b4d44a7..b2e6b48aa9 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
@@ -742,7 +742,7 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
    *
    * @param path path
    */
-  public IMNode getDeviceNodeWithAutoCreate(PartialPath path, boolean autoCreateSchema)
+  private IMNode getDeviceNodeWithAutoCreate(PartialPath path)
       throws IOException, MetadataException {
     IMNode node;
     try {
@@ -756,7 +756,7 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
       }
     } catch (Exception e) {
       if (e.getCause() instanceof MetadataException) {
-        if (!autoCreateSchema) {
+        if (!config.isAutoCreateSchemaEnabled()) {
           throw new PathNotExistException(path.getFullPath());
         }
       } else {
@@ -771,11 +771,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
     return node;
   }
 
-  public IMNode getDeviceNodeWithAutoCreate(PartialPath path)
-      throws MetadataException, IOException {
-    return getDeviceNodeWithAutoCreate(path, config.isAutoCreateSchemaEnabled());
-  }
-
   public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws MetadataException {
     IMNode node = mtree.getDeviceNodeWithAutoCreating(plan.getPath());
     mtree.unPinMNode(node);
@@ -1433,41 +1428,11 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
     PartialPath devicePath = plan.getDevicePath();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-    IMNode deviceMNode = null;
+    IMNode deviceMNode;
 
     // 1. get device node, set using template if accessed.
-    boolean mountedNodeFound = false;
-    boolean isDeviceInTemplate = false;
-    // check every measurement path
-    for (String measurementId : measurementList) {
-      PartialPath fullPath = devicePath.concatNode(measurementId);
-      int index = mtree.getMountedNodeIndexOnMeasurementPath(fullPath);
-      if ((index != fullPath.getNodeLength() - 1) && !mountedNodeFound) {
-        // this measurement is in template, need to assure mounted node exists and set using
-        // template.
-        // Without allowing overlap of template and MTree, this block run only once
-        String[] mountedPathNodes = Arrays.copyOfRange(fullPath.getNodes(), 0, index + 1);
-        IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
-        try {
-          if (!mountedNode.isUseTemplate()) {
-            mountedNode = setUsingSchemaTemplate(mountedNode);
-          }
-          mountedNodeFound = true;
-          if (index < devicePath.getNodeLength() - 1) {
-            deviceMNode =
-                mountedNode
-                    .getUpperTemplate()
-                    .getPathNodeInTemplate(
-                        new PartialPath(
-                            Arrays.copyOfRange(
-                                devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
-            isDeviceInTemplate = true;
-          }
-        } finally {
-          mtree.unPinMNode(mountedNode);
-        }
-      }
-    }
+    deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
+    boolean isDeviceInTemplate = deviceMNode != null;
     // get logical device node, may be in template. will be multiple if overlap is allowed.
     if (!isDeviceInTemplate) {
       deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
@@ -1551,6 +1516,43 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
     return deviceMNode;
   }
 
+  private IMNode getDeviceInTemplateIfUsingTemplate(
+      PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
+    // 1. get device node, set using template if accessed.
+    IMNode deviceMNode = null;
+
+    // check every measurement path
+    int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
+    if (index == devicePath.getNodeLength()) {
+      return null;
+    }
+
+    // this measurement is in template, need to assure mounted node exists and set using
+    // template.
+    // Without allowing overlap of template and MTree, this block run only once
+    String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
+    IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
+    try {
+      if (!mountedNode.isUseTemplate()) {
+        mountedNode = setUsingSchemaTemplate(mountedNode);
+      }
+    } finally {
+      mtree.unPinMNode(mountedNode);
+    }
+
+    if (index < devicePath.getNodeLength() - 1) {
+      deviceMNode =
+          mountedNode
+              .getUpperTemplate()
+              .getPathNodeInTemplate(
+                  new PartialPath(
+                      Arrays.copyOfRange(
+                          devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
+    }
+
+    return deviceMNode;
+  }
+
   private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
       InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
       throws MetadataException {
@@ -1597,19 +1599,23 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
   private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
       throws MetadataException {
     Template curTemplate = deviceMNode.getUpperTemplate();
-    if (curTemplate != null) {
-      IMeasurementSchema schema = curTemplate.getSchema(measurement);
-      if (!deviceMNode.isUseTemplate()) {
-        deviceMNode = setUsingSchemaTemplate(deviceMNode);
-      }
 
-      if (schema != null) {
-        return MeasurementMNode.getMeasurementMNode(
-            deviceMNode.getAsEntityMNode(), measurement, schema, null);
-      }
+    if (curTemplate == null) {
       return null;
     }
-    return null;
+
+    IMeasurementSchema schema = curTemplate.getSchema(measurement);
+
+    if (schema == null) {
+      return null;
+    }
+
+    if (!deviceMNode.isUseTemplate()) {
+      deviceMNode = setUsingSchemaTemplate(deviceMNode);
+    }
+
+    return MeasurementMNode.getMeasurementMNode(
+        deviceMNode.getAsEntityMNode(), measurement, schema, null);
   }
 
   /** create timeseries ignoring PathAlreadyExistException */