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 2022/09/08 08:14:18 UTC

[GitHub] [iotdb] Cpaulyz opened a new pull request, #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Cpaulyz opened a new pull request, #7270:
URL: https://github.com/apache/iotdb/pull/7270

   ## Description
   
   * Requirement docs: https://apache-iotdb.feishu.cn/docx/doxcnF9w5N6vxnQ9qak5zyMwoig


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


[GitHub] [iotdb] Cpaulyz commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
Cpaulyz commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r972212614


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java:
##########
@@ -142,7 +141,7 @@ public List<String> getAllDevicePatterns() {
   }
 
   private void searchDevicePattern(
-      PathPatternNode curNode, List<String> nodes, Set<String> results) {
+      PathPatternNode<Void> curNode, List<String> nodes, Set<String> results) {

Review Comment:
   I didn't change the semantics of this interface. It originally seemed to assume that the device of `root.sg1.d1.**` is `root.sg1.d1.**`.  I didn't know if the modification will have an impact on the upper layer application
   



##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java:
##########
@@ -26,46 +26,96 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
 
-public class PathPatternNode {
+import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class PathPatternNode<V> {
 
   private final String name;
-  private final Map<String, PathPatternNode> children;
+  private final Map<String, PathPatternNode<V>> children;
+  private Set<V> valueSet;

Review Comment:
   
   > It seems that you forgot about serializing this field.
   
   Nope. I reuse the data structure of the nodes in PathPatternTree here, mounting an additional valueSet. Serialization methods are only used in PathPatternTree, and the serialization information is used as a ByteBuffer in RPC. valueSet is always empty in PathPatternTree, so I did not serialize it to save network cost .
   
   
   > And equalsWIth may also take it into account
   
   I've added the valueSet comparison in `equalsWith`
   



##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.commons.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+
+@NotThreadSafe
+public class PatternTreeMap<V> {
+  private final PathPatternNode<V> root;
+  private final Supplier<? extends Set<V>> supplier;
+  private BiConsumer<V, Set<V>> appendFunction;
+  private BiConsumer<V, Set<V>> deleteFunction;

Review Comment:
   done



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


[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r971485066


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.commons.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+
+@NotThreadSafe
+public class PatternTreeMap<V> {
+  private final PathPatternNode<V> root;
+  private final Supplier<? extends Set<V>> supplier;
+  private BiConsumer<V, Set<V>> appendFunction;
+  private BiConsumer<V, Set<V>> deleteFunction;
+
+  /**
+   * Create PatternTreeMap.
+   *
+   * @param supplier provide type of set to store values on PathPatternNode
+   * @param appendFunction define the merge logic of append value
+   * @param deleteFunction define the split logic of delete value
+   */
+  public PatternTreeMap(
+      Supplier<? extends Set<V>> supplier,
+      BiConsumer<V, Set<V>> appendFunction,
+      BiConsumer<V, Set<V>> deleteFunction) {
+    this.root = new PathPatternNode<>(IoTDBConstant.PATH_ROOT, supplier);
+    this.supplier = supplier;
+    this.appendFunction = appendFunction;
+    this.deleteFunction = deleteFunction;
+  }
+
+  /**
+   * Append key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be appended
+   */
+  public void append(PartialPath key, V value) {
+    if (appendFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    String[] pathNodes = key.getNodes();
+    PathPatternNode<V> curNode = root;
+    for (int i = 1; i < pathNodes.length; i++) {
+      PathPatternNode<V> nextNode = curNode.getChildren(pathNodes[i]);
+      if (nextNode == null) {
+        nextNode = new PathPatternNode<>(pathNodes[i], supplier);
+        curNode.addChild(nextNode);
+      }
+      curNode = nextNode;
+    }
+    curNode.appendValue(value, appendFunction);
+  }
+
+  /**
+   * Delete key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be deleted
+   */
+  public void delete(PartialPath key, V value) {
+    if (deleteFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    deletePathNode(root, key.getNodes(), 0, value);
+  }
+
+  /**
+   * Recursive method for deleting value.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param value the value to be deleted
+   * @return true if current PathPatternNode can be removed
+   */
+  private boolean deletePathNode(PathPatternNode<V> node, String[] pathNodes, int pos, V value) {
+    if (node == null) {
+      return false;
+    }
+    if (pos == pathNodes.length - 1) {
+      node.deleteValue(value, deleteFunction);
+    } else {
+      PathPatternNode<V> child = node.getChildren(pathNodes[pos + 1]);
+      if (deletePathNode(child, pathNodes, pos + 1, value)) {
+        node.deleteChild(child);
+      }
+    }
+    return node.isLeaf() && node.getValues().isEmpty();
+  }
+
+  /**
+   * Get value list related to PathPattern that overlapped with fullPath.
+   *
+   * @param fullPath full path without wildcard
+   * @return value list
+   */
+  public List<V> getOverlapped(PartialPath fullPath) {
+    List<V> res = new ArrayList<>();
+    searchOverlapped(root, fullPath.getNodes(), 0, res, false);
+    return res;
+  }
+
+  /**
+   * Recursive method for search overlapped pattern.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param resultList result list
+   * @param fromMultiWildCard true if node in caller is '**', so there is no need to traverse the
+   *     remaining pathNodes
+   */
+  public void searchOverlapped(
+      PathPatternNode<V> node,
+      String[] pathNodes,
+      int pos,
+      List<V> resultList,
+      boolean fromMultiWildCard) {

Review Comment:
   `fromMultiWildCard` means there may exist two consecutive `**` in `PatternTreeMap`?
   I think this case should be deal with or avoided at the very beginning like during constructing this `PatternTreeMap`.



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


[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r971485312


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.commons.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+
+@NotThreadSafe
+public class PatternTreeMap<V> {
+  private final PathPatternNode<V> root;
+  private final Supplier<? extends Set<V>> supplier;
+  private BiConsumer<V, Set<V>> appendFunction;
+  private BiConsumer<V, Set<V>> deleteFunction;

Review Comment:
   ```suggestion
     private final BiConsumer<V, Set<V>> appendFunction;
     private final BiConsumer<V, Set<V>> deleteFunction;
   ```



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


[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r972584392


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java:
##########
@@ -142,7 +141,7 @@ public List<String> getAllDevicePatterns() {
   }
 
   private void searchDevicePattern(
-      PathPatternNode curNode, List<String> nodes, Set<String> results) {
+      PathPatternNode<Void> curNode, List<String> nodes, Set<String> results) {

Review Comment:
   You can discuss with @liuminghui233 and @MarcosZyk about it, I think it's a bug in previous implementation, and can be fixed by the way in this pr.



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


[GitHub] [iotdb] qiaojialin merged pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
qiaojialin merged PR #7270:
URL: https://github.com/apache/iotdb/pull/7270


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


[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r972584037


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java:
##########
@@ -26,46 +26,96 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
 
-public class PathPatternNode {
+import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class PathPatternNode<V> {
 
   private final String name;
-  private final Map<String, PathPatternNode> children;
+  private final Map<String, PathPatternNode<V>> children;
+  private Set<V> valueSet;

Review Comment:
   I think it's better to do the serialization in PathPatternNode, because `PatternTreeMap` also need to be serialized and tranferred by rpc.
   It's fine to occupy four more bytes(int size) for one node to indicate the valueSet's size, one packet in network will be like 1KB or something else, four more bytes per node will not hurt the perfermance.



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


[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r971459361


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java:
##########
@@ -26,46 +26,96 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
 
-public class PathPatternNode {
+import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class PathPatternNode<V> {
 
   private final String name;
-  private final Map<String, PathPatternNode> children;
+  private final Map<String, PathPatternNode<V>> children;
+  private Set<V> valueSet;

Review Comment:
   It seems that you forgot about serializing this field.



##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.commons.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+
+@NotThreadSafe
+public class PatternTreeMap<V> {
+  private final PathPatternNode<V> root;
+  private final Supplier<? extends Set<V>> supplier;
+  private BiConsumer<V, Set<V>> appendFunction;
+  private BiConsumer<V, Set<V>> deleteFunction;
+
+  /**
+   * Create PatternTreeMap.
+   *
+   * @param supplier provide type of set to store values on PathPatternNode
+   * @param appendFunction define the merge logic of append value
+   * @param deleteFunction define the split logic of delete value
+   */
+  public PatternTreeMap(
+      Supplier<? extends Set<V>> supplier,
+      BiConsumer<V, Set<V>> appendFunction,
+      BiConsumer<V, Set<V>> deleteFunction) {
+    this.root = new PathPatternNode<>(IoTDBConstant.PATH_ROOT, supplier);
+    this.supplier = supplier;
+    this.appendFunction = appendFunction;
+    this.deleteFunction = deleteFunction;
+  }
+
+  /**
+   * Append key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be appended
+   */
+  public void append(PartialPath key, V value) {
+    if (appendFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    String[] pathNodes = key.getNodes();
+    PathPatternNode<V> curNode = root;
+    for (int i = 1; i < pathNodes.length; i++) {
+      PathPatternNode<V> nextNode = curNode.getChildren(pathNodes[i]);
+      if (nextNode == null) {
+        nextNode = new PathPatternNode<>(pathNodes[i], supplier);
+        curNode.addChild(nextNode);
+      }
+      curNode = nextNode;
+    }
+    curNode.appendValue(value, appendFunction);
+  }
+
+  /**
+   * Delete key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be deleted
+   */
+  public void delete(PartialPath key, V value) {
+    if (deleteFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    deletePathNode(root, key.getNodes(), 0, value);
+  }
+
+  /**
+   * Recursive method for deleting value.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param value the value to be deleted
+   * @return true if current PathPatternNode can be removed
+   */
+  private boolean deletePathNode(PathPatternNode<V> node, String[] pathNodes, int pos, V value) {
+    if (node == null) {
+      return false;
+    }
+    if (pos == pathNodes.length - 1) {
+      node.deleteValue(value, deleteFunction);
+    } else {
+      PathPatternNode<V> child = node.getChildren(pathNodes[pos + 1]);
+      if (deletePathNode(child, pathNodes, pos + 1, value)) {
+        node.deleteChild(child);
+      }
+    }
+    return node.isLeaf() && node.getValues().isEmpty();
+  }
+
+  /**
+   * Get value list related to PathPattern that overlapped with fullPath.
+   *
+   * @param fullPath full path without wildcard
+   * @return value list
+   */
+  public List<V> getOverlapped(PartialPath fullPath) {
+    List<V> res = new ArrayList<>();
+    searchOverlapped(root, fullPath.getNodes(), 0, res, false);
+    return res;
+  }
+
+  /**
+   * Recursive method for search overlapped pattern.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param resultList result list
+   * @param fromMultiWildCard true if node in caller is '**', so there is no need to traverse the
+   *     remaining pathNodes
+   */
+  public void searchOverlapped(
+      PathPatternNode<V> node,
+      String[] pathNodes,
+      int pos,
+      List<V> resultList,
+      boolean fromMultiWildCard) {

Review Comment:
   `fromMultiWildCard` means there may exist two consecutive `**` in `PathPatternTree`?
   I think this case should be deal with or avoided at the very beginning like during constructing this `PathPatternTree`.



##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java:
##########
@@ -142,7 +141,7 @@ public List<String> getAllDevicePatterns() {
   }
 
   private void searchDevicePattern(
-      PathPatternNode curNode, List<String> nodes, Set<String> results) {
+      PathPatternNode<Void> curNode, List<String> nodes, Set<String> results) {

Review Comment:
   What if `root.sg.d1.**`, it seems that this method won't put it into result.



##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java:
##########
@@ -26,46 +26,96 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
 
-public class PathPatternNode {
+import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class PathPatternNode<V> {
 
   private final String name;
-  private final Map<String, PathPatternNode> children;
+  private final Map<String, PathPatternNode<V>> children;
+  private Set<V> valueSet;

Review Comment:
   And equalsWIth may also take it into account



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


[GitHub] [iotdb] Cpaulyz commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
Cpaulyz commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r971609919


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.commons.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+
+@NotThreadSafe
+public class PatternTreeMap<V> {
+  private final PathPatternNode<V> root;
+  private final Supplier<? extends Set<V>> supplier;
+  private BiConsumer<V, Set<V>> appendFunction;
+  private BiConsumer<V, Set<V>> deleteFunction;
+
+  /**
+   * Create PatternTreeMap.
+   *
+   * @param supplier provide type of set to store values on PathPatternNode
+   * @param appendFunction define the merge logic of append value
+   * @param deleteFunction define the split logic of delete value
+   */
+  public PatternTreeMap(
+      Supplier<? extends Set<V>> supplier,
+      BiConsumer<V, Set<V>> appendFunction,
+      BiConsumer<V, Set<V>> deleteFunction) {
+    this.root = new PathPatternNode<>(IoTDBConstant.PATH_ROOT, supplier);
+    this.supplier = supplier;
+    this.appendFunction = appendFunction;
+    this.deleteFunction = deleteFunction;
+  }
+
+  /**
+   * Append key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be appended
+   */
+  public void append(PartialPath key, V value) {
+    if (appendFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    String[] pathNodes = key.getNodes();
+    PathPatternNode<V> curNode = root;
+    for (int i = 1; i < pathNodes.length; i++) {
+      PathPatternNode<V> nextNode = curNode.getChildren(pathNodes[i]);
+      if (nextNode == null) {
+        nextNode = new PathPatternNode<>(pathNodes[i], supplier);
+        curNode.addChild(nextNode);
+      }
+      curNode = nextNode;
+    }
+    curNode.appendValue(value, appendFunction);
+  }
+
+  /**
+   * Delete key and value to PatternTreeMap.
+   *
+   * @param key PartialPath that can contain '*' or '**'
+   * @param value The value to be deleted
+   */
+  public void delete(PartialPath key, V value) {
+    if (deleteFunction == null) {
+      throw new UnsupportedOperationException();
+    }
+    deletePathNode(root, key.getNodes(), 0, value);
+  }
+
+  /**
+   * Recursive method for deleting value.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param value the value to be deleted
+   * @return true if current PathPatternNode can be removed
+   */
+  private boolean deletePathNode(PathPatternNode<V> node, String[] pathNodes, int pos, V value) {
+    if (node == null) {
+      return false;
+    }
+    if (pos == pathNodes.length - 1) {
+      node.deleteValue(value, deleteFunction);
+    } else {
+      PathPatternNode<V> child = node.getChildren(pathNodes[pos + 1]);
+      if (deletePathNode(child, pathNodes, pos + 1, value)) {
+        node.deleteChild(child);
+      }
+    }
+    return node.isLeaf() && node.getValues().isEmpty();
+  }
+
+  /**
+   * Get value list related to PathPattern that overlapped with fullPath.
+   *
+   * @param fullPath full path without wildcard
+   * @return value list
+   */
+  public List<V> getOverlapped(PartialPath fullPath) {
+    List<V> res = new ArrayList<>();
+    searchOverlapped(root, fullPath.getNodes(), 0, res, false);
+    return res;
+  }
+
+  /**
+   * Recursive method for search overlapped pattern.
+   *
+   * @param node current PathPatternNode
+   * @param pathNodes pathNodes of key
+   * @param pos current index of pathNodes
+   * @param resultList result list
+   * @param fromMultiWildCard true if node in caller is '**', so there is no need to traverse the
+   *     remaining pathNodes
+   */
+  public void searchOverlapped(
+      PathPatternNode<V> node,
+      String[] pathNodes,
+      int pos,
+      List<V> resultList,
+      boolean fromMultiWildCard) {

Review Comment:
   `fromMultiWildCard` is unnecessary. I will remove it later.



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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7270: [IOTDB-4335] Implement PathPatternTree to support trigger and delete

Posted by GitBox <gi...@apache.org>.
MarcosZyk commented on code in PR #7270:
URL: https://github.com/apache/iotdb/pull/7270#discussion_r973518167


##########
node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java:
##########
@@ -142,7 +141,7 @@ public List<String> getAllDevicePatterns() {
   }
 
   private void searchDevicePattern(
-      PathPatternNode curNode, List<String> nodes, Set<String> results) {
+      PathPatternNode<Void> curNode, List<String> nodes, Set<String> results) {

Review Comment:
   The device of ```root.sg1.d1.**``` may be  ```root.sg1.d1``` and ```root.sg1.d1.**```.



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