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 2021/09/17 02:34:05 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #3918: [IOTDB-1026] Support wildcard ** in Path And Replace PrefixPath usage with PathPattern in IOTDB-SQL

qiaojialin commented on a change in pull request #3918:
URL: https://github.com/apache/iotdb/pull/3918#discussion_r710073848



##########
File path: cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
##########
@@ -155,7 +155,7 @@ public void testWithoutCreateSchema() throws IOException, ClassNotFoundException
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select * from root")) {
+      if (statement.execute("select * from root.**")) {

Review comment:
       change all `select * from root.**` to `select ** from root`

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
##########
@@ -1204,14 +1204,14 @@ public void setCoordinator(Coordinator coordinator) {
   /** Similar to method getAllTimeseriesPath(), but return Path with alias alias. */
   @Override
   public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
-      PartialPath prefixPath, int limit, int offset) throws MetadataException {
+      PartialPath path, int limit, int offset) throws MetadataException {
 
     // get all storage groups this path may belong to
     // the key is the storage group name and the value is the path to be queried with storage group
     // added, e.g:
     // "root.*" will be translated into:
     // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...

Review comment:
       I cannot understand this...

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
##########
@@ -136,24 +140,49 @@ public PartialPath alterPrefixPath(PartialPath prefixPath) {
   }
 
   /**
-   * Test if this PartialPath matches a full path. rPath is supposed to be a full timeseries path
-   * without wildcards. e.g. "root.sg.device.*" matches path "root.sg.device.s1" whereas it does not
-   * match "root.sg.device" and "root.sg.vehicle.s1"
+   * Test if this PartialPath matches a full path. This partialPath acts as a full path pattern.
+   * rPath is supposed to be a full timeseries path without wildcards. e.g. "root.sg.device.*"
+   * matches path "root.sg.device.s1" whereas it does not match "root.sg.device" and
+   * "root.sg.vehicle.s1"
    *
    * @param rPath a plain full path of a timeseries
    * @return true if a successful match, otherwise return false
    */
   public boolean matchFullPath(PartialPath rPath) {
-    String[] rNodes = rPath.getNodes();
-    if (rNodes.length < nodes.length) {
+    return matchFullPath(rPath.getNodes(), 0, 0, false);
+  }
+
+  private boolean matchFullPath(
+      String[] pathNodes, int pathIndex, int patternIndex, boolean multiLevelWild) {
+    if (pathIndex == pathNodes.length && patternIndex == nodes.length) {
+      return true;
+    } else if (patternIndex == nodes.length && multiLevelWild) {
+      return matchFullPath(pathNodes, pathIndex + 1, patternIndex, true);
+    } else if (pathIndex >= pathNodes.length || patternIndex >= nodes.length) {
       return false;
     }
-    for (int i = 0; i < nodes.length; i++) {
-      if (!nodes[i].equals(IoTDBConstant.PATH_WILDCARD) && !nodes[i].equals(rNodes[i])) {
-        return false;
+
+    String pathNode = pathNodes[pathIndex];
+    String patternNode = nodes[patternIndex];
+    boolean isMatch = false;
+    if (patternNode.equals(PATH_MULTI_LEVEL_WILDCARD)) {
+      isMatch = matchFullPath(pathNodes, pathIndex + 1, patternIndex + 1, true);
+    } else {
+      if (patternNode.contains(PATH_ONE_LEVEL_WILDCARD)) {

Review comment:
       add a test case:
   root.sg.device*

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -797,35 +799,42 @@ public TSDataType getSeriesType(PartialPath path) throws MetadataException {
   /**
    * Get all devices under given prefixPath.
    *
-   * @param prefixPath a prefix of a full path. if the wildcard is not at the tail, then each
-   *     wildcard can only match one level, otherwise it can match to the tail.
+   * @param prefixPath a prefix of a full path or a pattern of the prefixPath.
    * @return A HashSet instance which stores devices paths with given prefixPath.
    */
-  public Set<PartialPath> getDevices(PartialPath prefixPath) throws MetadataException {
-    return mtree.getDevices(prefixPath);
+  public Set<PartialPath> getDevicesByPrefix(PartialPath prefixPath) throws MetadataException {
+    return mtree.getDevices(prefixPath, true);
+  }
+
+  public Set<PartialPath> getDevicesForTimeseries(PartialPath timeseries) throws MetadataException {
+    return mtree.getDevicesForTimeseries(timeseries);
+  }
+
+  public Set<PartialPath> getDevices(PartialPath pathPattern) throws MetadataException {

Review comment:
       could we separate pathPattern from PartialPath by define two class?

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
##########
@@ -115,7 +115,8 @@ private IoTDBConstant() {}
   public static final String COLUMN_TRIGGER_STATUS_STARTED = "started";
   public static final String COLUMN_TRIGGER_STATUS_STOPPED = "stopped";
 
-  public static final String PATH_WILDCARD = "*";
+  public static final String PATH_ONE_LEVEL_WILDCARD = "*";
+  public static final String PATH_MULTI_LEVEL_WILDCARD = "**";

Review comment:
       ```suggestion
     public static final String MULTI_LEVEL_PATH_WILDCARD = "**";
   ```

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
##########
@@ -1204,14 +1204,14 @@ public void setCoordinator(Coordinator coordinator) {
   /** Similar to method getAllTimeseriesPath(), but return Path with alias alias. */
   @Override
   public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
-      PartialPath prefixPath, int limit, int offset) throws MetadataException {
+      PartialPath path, int limit, int offset) throws MetadataException {
 
     // get all storage groups this path may belong to
     // the key is the storage group name and the value is the path to be queried with storage group
     // added, e.g:
     // "root.*" will be translated into:
     // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...
-    Map<String, String> sgPathMap = determineStorageGroup(prefixPath);
+    Map<String, String> sgPathMap = determineStorageGroup(path);

Review comment:
       better to change determinStorageGroup to getStorageGroups or so

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -797,35 +799,42 @@ public TSDataType getSeriesType(PartialPath path) throws MetadataException {
   /**
    * Get all devices under given prefixPath.
    *
-   * @param prefixPath a prefix of a full path. if the wildcard is not at the tail, then each
-   *     wildcard can only match one level, otherwise it can match to the tail.
+   * @param prefixPath a prefix of a full path or a pattern of the prefixPath.
    * @return A HashSet instance which stores devices paths with given prefixPath.
    */
-  public Set<PartialPath> getDevices(PartialPath prefixPath) throws MetadataException {
-    return mtree.getDevices(prefixPath);
+  public Set<PartialPath> getDevicesByPrefix(PartialPath prefixPath) throws MetadataException {
+    return mtree.getDevices(prefixPath, true);
+  }
+
+  public Set<PartialPath> getDevicesForTimeseries(PartialPath timeseries) throws MetadataException {

Review comment:
       add javadoc, can not get the meaning of this method, getDevicesByTimeseries?

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
##########
@@ -115,7 +115,8 @@ private IoTDBConstant() {}
   public static final String COLUMN_TRIGGER_STATUS_STARTED = "started";
   public static final String COLUMN_TRIGGER_STATUS_STOPPED = "stopped";
 
-  public static final String PATH_WILDCARD = "*";
+  public static final String PATH_ONE_LEVEL_WILDCARD = "*";

Review comment:
       ```suggestion
     public static final String ONE_LEVEL_PATH_WILDCARD = "*";
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.mtree.traverser;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_MULTI_LEVEL_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ONE_LEVEL_WILDCARD;
+
+// This class defines the main traversal framework and declares some methods for result process
+// extension.
+public abstract class Traverser {
+
+  protected IMNode startNode;
+  protected String[] nodes;
+
+  // if isMeasurementTraverser, measurement in template should be processed
+  protected boolean isMeasurementTraverser = false;
+
+  // default false means fullPath pattern match
+  protected boolean isPrefixMatch = false;
+
+  // level query option
+  protected boolean isLevelTraverser = false;
+  protected int targetLevel;
+
+  // traverse for specific storage group
+  protected StorageGroupFilter storageGroupFilter = null;
+
+  public Traverser(IMNode startNode, PartialPath path) throws MetadataException {
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(startNode.getName())) {
+      throw new IllegalPathException(
+          path.getFullPath(), path.getFullPath() + " doesn't start with " + startNode.getName());
+    }
+    this.startNode = startNode;
+    this.nodes = nodes;
+  }
+
+  public void traverse() throws MetadataException {

Review comment:
       add javadoc, How to get the result.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.mtree.traverser;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_MULTI_LEVEL_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ONE_LEVEL_WILDCARD;
+
+// This class defines the main traversal framework and declares some methods for result process
+// extension.
+public abstract class Traverser {
+
+  protected IMNode startNode;
+  protected String[] nodes;
+
+  // if isMeasurementTraverser, measurement in template should be processed
+  protected boolean isMeasurementTraverser = false;
+
+  // default false means fullPath pattern match
+  protected boolean isPrefixMatch = false;
+
+  // level query option
+  protected boolean isLevelTraverser = false;
+  protected int targetLevel;
+
+  // traverse for specific storage group
+  protected StorageGroupFilter storageGroupFilter = null;
+
+  public Traverser(IMNode startNode, PartialPath path) throws MetadataException {
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(startNode.getName())) {
+      throw new IllegalPathException(
+          path.getFullPath(), path.getFullPath() + " doesn't start with " + startNode.getName());
+    }
+    this.startNode = startNode;
+    this.nodes = nodes;
+  }
+
+  public void traverse() throws MetadataException {
+    traverse(startNode, 0, false, 0);
+  }
+
+  /**
+   * The recursive method for MTree traversal.
+   *
+   * @param node current node that match the targetName in given path
+   * @param idx the index of targetName in given path
+   * @param multiLevelWildcard whether the current targetName is **
+   * @param level the level of current node in MTree
+   * @throws MetadataException some result process may throw MetadataException
+   */
+  protected void traverse(IMNode node, int idx, boolean multiLevelWildcard, int level)

Review comment:
       add javadoc:  if the node matches nodes[idx], then do some operation and traverse the children with nodes[idx+1]
   
   // intermal match: root.sg internal match root.sg.**(pattern)
   boolean processInternalMatchedMNode(IMNode node)
   
   // full match: root.sg.d full match root.sg.**(pattern)
   boolean processFullMatchedMNode(IMNode node)

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
##########
@@ -32,6 +32,10 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.regex.Pattern;
+

Review comment:
       maybe changing PartialPath to PathPattern is better




-- 
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