You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/09/08 10:17:53 UTC

[GitHub] [pinot] Ferrari6 opened a new pull request #7409: Fix JSONPath cache inefficient issue

Ferrari6 opened a new pull request #7409:
URL: https://github.com/apache/pinot/pull/7409


   ## Description
   this commit fixes #7403 
   
   ### Background
   > When we used jsonpath transformation functions, we found that there was a delay in consumption, and the CPU usage was very high. Analysis of jstack found that the consumption threads were waiting for the lock of LRUCache in jayway, and further analysis of the CPU and lock contented, we can confirm that this inefficient LRUCache is the consumption performance bottleneck.
   
   **stack trace**
   ![image](https://user-images.githubusercontent.com/89431368/132486165-b1d0680f-9bf2-423b-9dcb-31afd3a5e8e4.png)
   
    _**flamegraphs can be found in the issue descriptions #7403 **_
   
   ### Fix
   A new JSON path cache is implemented using ConcurrentHashMap, and the cache threshold is set at the same time. When the maximum is exceeded, the JSON path will not be cached anymore.
   
   - In Pinot, the number of JSON paths is bounded by the size of the transformation config
   - Even if it exceeds the maximum cache size, not cache JSON path may be better than frequent swapping in and out of LRU
   - If JSON path compile is not cached, CPU consumption is also very small
   
   ### 
   
   ```
   "transformConfigs": [
           {
             "columnName": "id",
             "transformFunction": "jsonPathString(report,'$.identifiers.id','')"
           },
          {
             "columnName": "name",
             "transformFunction": "jsonPathString(report,'$.identifiers.name','')"
           },
    ...
   ]
   ```
   
   
   ### Pinot Server Flamegraphs when using ConcurrentHashMap cache (28vcpu)
   
   ![image](https://user-images.githubusercontent.com/89431368/132490363-43887dfc-895c-4569-9228-e135149f9f9c.png)
   ![image](https://user-images.githubusercontent.com/89431368/132490522-d0faca18-ff77-4afa-a80c-2a0c7583eb8b.png)
   **jsonpath CPU usage is low and no lock contentions**
   -->
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   <!-- If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release. -->
   
   <!-- If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text.
   -->
   ## Documentation
   <!-- If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   -->
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705168129



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/FunctionConstants.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.pinot.common.function;
+
+public class FunctionConstants {
+  private FunctionConstants() {
+  }
+
+  /**
+   * Maximum of json path cache size.
+   * the number of JSON paths is bounded by the size of the transformation config,
+   * add this threshold for protection from the extreme cases.
+   */
+  public static final int MAX_JSON_PATH_CACHE_SIZE = 1024 * 16;

Review comment:
       This class should be removed and the constant inlined into `JsonPathMapCache` where it is relevant. 




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705601492



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128);
+  private final int _limit = 1024 * 16;

Review comment:
       Make this limit a constant
   ```suggestion
     private static final int LIMIT = 1024 * 16;
   ```

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128);
+  private final int _limit = 1024 * 16;
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {
+      // use putIfAbsent to handle concurrency put

Review comment:
       `put()` is also thread safe fyi. I think it will only try to put the value when there is a cache miss?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       JsonPath can also be used at query time (see `JsonExtractScalarTransformFunction`). Let's add a TODO here to add some evict policy in the future

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128);
+  private final int _limit = 1024 * 16;

Review comment:
       Should we consider using a limit that takes the scale factor (0.75) into consideration? e.g. use 12000 so that the capacity of the map is bounded to 1024*16

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128, 0.75f, 1);
+
+  private final int _limit;
+
+  public JsonPathMapCache(int limit) {
+    this._limit = limit;
+  }
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {

Review comment:
       (code format)
   ```suggestion
       if (size() < LIMIT) {
   ```




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang merged pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #7409:
URL: https://github.com/apache/pinot/pull/7409


   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-920437691


   > @Ferrari6 Can you rebase based on the master branch and retrigger github action tests? We recently introduced the flakyness to one of our integration test and fixed from #7432
   
   yes, sure. But because the setCache of `CacheProvider` in Jayway can only be called once, and once `getCache` is called, it cannot be `setCache` again. So I need to do some code changes.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (fd7aed0) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `56.99%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7409       +/-   ##
   =============================================
   - Coverage     71.54%   14.55%   -57.00%     
   + Complexity     4036       80     -3956     
   =============================================
     Files          1579     1534       -45     
     Lines         80390    78512     -1878     
     Branches      11945    11741      -204     
   =============================================
   - Hits          57512    11424    -46088     
   - Misses        18996    66249    +47253     
   + Partials       3882      839     -3043     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.55% <0.00%> (-0.05%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `0.00% <0.00%> (-81.36%)` | :arrow_down: |
   | [...form/function/JsonExtractKeyTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RLZXlUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-74.08%)` | :arrow_down: |
   | [...m/function/JsonExtractScalarTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RTY2FsYXJUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1254 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...fd7aed0](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (fd7aed0) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `6.43%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.54%   65.10%   -6.44%     
   + Complexity     4036     4033       -3     
   ============================================
     Files          1579     1534      -45     
     Lines         80390    78512    -1878     
     Branches      11945    11741     -204     
   ============================================
   - Hits          57512    51115    -6397     
   - Misses        18996    23746    +4750     
   + Partials       3882     3651     -231     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `68.61% <80.00%> (-0.05%)` | :arrow_down: |
   | unittests2 | `14.55% <0.00%> (-0.05%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...m/function/JsonExtractScalarTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RTY2FsYXJUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `43.80% <50.00%> (-6.20%)` | :arrow_down: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `85.71% <85.71%> (ø)` | |
   | [...form/function/JsonExtractKeyTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RLZXlUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `73.91% <92.30%> (-0.17%)` | :arrow_down: |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `80.35% <100.00%> (-1.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [364 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...fd7aed0](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (fd7aed0) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `1.27%`.
   > The diff coverage is `82.85%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.54%   70.26%   -1.28%     
   + Complexity     4036     4033       -3     
   ============================================
     Files          1579     1580       +1     
     Lines         80390    80386       -4     
     Branches      11945    11944       -1     
   ============================================
   - Hits          57512    56482    -1030     
   - Misses        18996    20025    +1029     
   + Partials       3882     3879       -3     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.67% <48.57%> (-0.09%)` | :arrow_down: |
   | unittests1 | `68.61% <80.00%> (-0.05%)` | :arrow_down: |
   | unittests2 | `14.55% <0.00%> (-0.05%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...m/function/JsonExtractScalarTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RTY2FsYXJUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `43.80% <50.00%> (-6.20%)` | :arrow_down: |
   | [...form/function/JsonExtractKeyTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RLZXlUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `73.91% <92.30%> (-0.17%)` | :arrow_down: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `80.35% <100.00%> (-1.00%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [...ot/common/messages/RoutingTableRebuildMessage.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvUm91dGluZ1RhYmxlUmVidWlsZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-81.82%)` | :arrow_down: |
   | ... and [109 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...fd7aed0](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930146952


   > Some tests failed, rerunning to see if intermittent issue due to timeouts.
   
   Thanks @mayankshriv. I have fixed the failed tests.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c03156c) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `40.79%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7409       +/-   ##
   =============================================
   - Coverage     71.54%   30.74%   -40.80%     
   =============================================
     Files          1579     1571        -8     
     Lines         80390    80055      -335     
     Branches      11945    11907       -38     
   =============================================
   - Hits          57512    24613    -32899     
   - Misses        18996    53327    +34331     
   + Partials       3882     2115     -1767     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.05% <92.30%> (-0.20%)` | :arrow_down: |
   | integration2 | `27.65% <100.00%> (-0.10%)` | :arrow_down: |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `84.37% <100.00%> (-2.73%)` | :arrow_down: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/config/table/TableType.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/DimensionFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EaW1lbnNpb25GaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/FileFormat.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0ZpbGVGb3JtYXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1067 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...c03156c](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705871929



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       I think guava `Cache` is a more safe cache solution cause the `JsonPath` cache will also be applied to the query.
   For one of my table ingestion cases, reading data QPS close to 10w and configuring `JsonPath` transformations with dozens of fields, this will put a certain pressure on GC and CPU because of the LRU algorithm needs to record every `get`. Of course, this should not become a system bottleneck, but the implementation is a bit too heavy.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] snleee commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
snleee commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-920374117


   @Ferrari6 Can you rebase based on the master branch and retrigger github action tests?


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930341016


   w
   
   > > Some tests failed, rerunning to see if intermittent issue due to timeouts.
   > 
   > Thanks @mayankshriv. I have fixed the failed tests.
   
   @Ferrari6 seems like there are still failing tests.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705099616



##########
File path: pinot-common/src/test/java/org/apache/pinot/common/function/JsonPathMapCacheTest.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.common.function;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Predicate;
+import com.jayway.jsonpath.internal.Utils;
+import com.jayway.jsonpath.spi.cache.Cache;
+import com.jayway.jsonpath.spi.cache.CacheProvider;
+import java.util.Collections;
+import java.util.LinkedList;
+import org.apache.pinot.common.function.scalar.JsonFunctions;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+
+public class JsonPathMapCacheTest {
+
+  private String _jsonString;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    _jsonString = "{\"name\":\"Jack\",\"contact\":{\"phone\":\"01234567\",\"email\":\"test@example.com\"}}";
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheUsed() throws JsonProcessingException {
+    // request json path function, verify SimpleJsonPathMapCache used
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, "$.contact.email"), "test@example.com");
+    Cache cache = CacheProvider.getCache();
+    assertEquals(cache.getClass(), JsonPathMapCache.class);
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheWorks() throws JsonProcessingException {
+    String path = "$.contact.email";
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, path), "test@example.com");
+    JsonPathMapCache cache = (JsonPathMapCache) CacheProvider.getCache();
+
+    // verify json path has been cached
+    LinkedList<Predicate> filterStack = new LinkedList<>(Collections.emptyList());
+    String cacheKey = Utils.concat(path, filterStack.toString());
+    JsonPath jsonPath = cache.get(cacheKey);
+    assertNotNull(jsonPath);
+  }

Review comment:
       Good suggestions. 
   I always prefer to use Mockitio for unit testing, but all method calls related to JSONFunctions are static, so I try to check the map size now.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705839273



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128);
+  private final int _limit = 1024 * 16;
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {
+      // use putIfAbsent to handle concurrency put

Review comment:
       Both are safe of these two puts, but there will be duplicated put when a cache misses because there is no mutex lock from get to put. I think putIfAbsent can reduce one unnecessary Map node value assignment. I think it may doesn't matter.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705396060



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128, 0.75f, 1);

Review comment:
       I will change it.  This is the habit of my previous team!




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv removed a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv removed a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930284631


   @Ferrari6 There's unit test failures (I reran a couple of times). Do you mind investigating? cc: @richardstartin 


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705620782



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       I think in the future this can be handled better by just precompiling `JsonPath` objects and bypassing the library's cache entirely. For transformConfig, it would require changes to the way `ScalarFunction` is defined as well as changes to `InbuiltFunctionEvaluator.planExecution` (this would also permit validation and cleansing of config, which is currently impossible). For `JsonExtractScalarTransformFunction` the change is trivial. 
   
   I think in the meantime using a guava `Cache` instead of a `ConcurrentHashMap` as suggested on slack would alleviate this concern.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang merged pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #7409:
URL: https://github.com/apache/pinot/pull/7409


   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] ben-manes commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r710760248



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       You will likely be fine using Guava if you increase the concurrency level (e.g. to 64, defaults to 4). That will stripe the LRU so that there is less contention, but will still suffer at peak loads ([benchmarks](https://github.com/ben-manes/caffeine/wiki/Benchmarks#read-100-1)). For your use-case here that is likely good enough given other costs, e.g. JsonPath's [inefficiencies](https://github.com/json-path/JsonPath/issues/740). Unfortunately my patches to Guava that gave it a ~25x boost were not accepted due to a lack of a champion, and instead they simply updated the JavaDoc to recommend Caffeine. 🙂 




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] ben-manes commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r711188514



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       It's not a problem. I co-authored Guava's so either way my code is used. 😄 
   
   @Ferrari6 is correct in what your preference and default should be, though. Prior to my involvement, the Guava team bet on reference caching (MapMaker) and spent their complexity budget by forking the hash table as an optimization. This was a mistake because soft references can cause GC death spirals of stop the world events and unpredictable evictions, but looked fine in a naive benchmark. This blew their complexity budget, so porting size eviction from [CLHM](https://github.com/ben-manes/concurrentlinkedhashmap) favored simplicity over performance.
   
   The longer-term problem you'll face is that no one maintains CacheBuilder. The last big change was adding Map.compute, but it was riddled with major bugs and done inefficiently. I fixed some of those problems, but there are show stoppers. If you keep to the historic functionality then Guava's can be made acceptable in most cases. Caffeine does have jar bloat by code generating per-configuration entry classes to minimize the memory footprint. In those cases where disk is a premium some projects embed CLHM into their code base, e.g. msjdbc and groovy. You'll probably have many caches throughout Pinot making the Caffeine dependency worthwhile even if you can get by in a case-by-case basis.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705391729



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128, 0.75f, 1);
+
+  private final int _limit;
+
+  public JsonPathMapCache(int limit) {
+    this._limit = limit;
+  }
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {

Review comment:
       Yes, it doesn't need a hard bound.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r710797948



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       You are right. 




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-920443797


   Both `JsonFunctions` and `JsonExtractScalarTransformFunction` use JsonPath, so need to set Jayway's `CacheProvider` before the initialization of these two functions and can only be set once. At the same time, I am also doing some performance tests on `Guava Cache`, I will submit some more code soon.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-929855328






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930146952


   > Some tests failed, rerunning to see if intermittent issue due to timeouts.
   
   Thanks @mayankshriv. I have fixed the failed tests.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang merged pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #7409:
URL: https://github.com/apache/pinot/pull/7409






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705932231



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       Guava Cache uses `recencyQueue` to record cache accesses. A large number of `CAS` enqueue operations generated by hotkeys will cause CPU consumption, and this synchronous recording will also slow down cache reading.
   
   I think maybe [Caffeine Cache](https://github.com/ben-manes/caffeine) is a good choice. Its Striped-RingBuffer design is more efficient for GC and CPU usage. And its API is very similar to Guava Cache.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (45387e0) into [master](https://codecov.io/gh/apache/pinot/commit/e7901ee8f72c0650dc348e5f9b3cdef7df1566e0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e7901ee) will **decrease** coverage by `2.13%`.
   > The diff coverage is `97.95%`.
   
   > :exclamation: Current head 45387e0 differs from pull request most recent head da2a5a0. Consider uploading reports for the commit da2a5a0 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.84%   69.70%   -2.14%     
   + Complexity     3344     3255      -89     
   ============================================
     Files          1517     1124     -393     
     Lines         75039    52970   -22069     
     Branches      10921     7985    -2936     
   ============================================
   - Hits          53910    36922   -16988     
   + Misses        17504    13425    -4079     
   + Partials       3625     2623    -1002     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `69.70% <97.95%> (+0.01%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/JsonPathMapCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhNYXBDYWNoZS5qYXZh) | `87.50% <87.50%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `81.03% <100.00%> (+0.33%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `79.25% <100.00%> (+0.68%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [597 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e7901ee...da2a5a0](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r710773435



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       I think we should use the default value of the `concurrencyLevel` of `Guava cache`. In this scenario, there will be very few caches writes and no need to update the cache. My concern is the way Guava records cache reads, which is not very efficient. 




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705620782



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       I think in the future this can be handled better by just precompiling `JsonPath` objects and bypassing the library's cache entirely. For transformConfig, it would require changes to the way `ScalarFunction` is defined as well as changes to `InbuiltFunctionEvaluator.planExecution`. For `JsonExtractScalarTransformFunction` the change is trivial. 
   
   I think in the meantime using a guava `Cache` instead of a `ConcurrentHashMap` as suggested on slack would alleviate this concern.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r704596924



##########
File path: pinot-common/src/test/java/org/apache/pinot/common/function/JsonPathMapCacheTest.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.common.function;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Predicate;
+import com.jayway.jsonpath.internal.Utils;
+import com.jayway.jsonpath.spi.cache.Cache;
+import com.jayway.jsonpath.spi.cache.CacheProvider;
+import java.util.Collections;
+import java.util.LinkedList;
+import org.apache.pinot.common.function.scalar.JsonFunctions;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+
+public class JsonPathMapCacheTest {
+
+  private String _jsonString;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    _jsonString = "{\"name\":\"Jack\",\"contact\":{\"phone\":\"01234567\",\"email\":\"test@example.com\"}}";
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheUsed() throws JsonProcessingException {
+    // request json path function, verify SimpleJsonPathMapCache used
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, "$.contact.email"), "test@example.com");
+    Cache cache = CacheProvider.getCache();
+    assertEquals(cache.getClass(), JsonPathMapCache.class);
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheWorks() throws JsonProcessingException {
+    String path = "$.contact.email";
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, path), "test@example.com");
+    JsonPathMapCache cache = (JsonPathMapCache) CacheProvider.getCache();
+
+    // verify json path has been cached
+    LinkedList<Predicate> filterStack = new LinkedList<>(Collections.emptyList());
+    String cacheKey = Utils.concat(path, filterStack.toString());
+    JsonPath jsonPath = cache.get(cacheKey);
+    assertNotNull(jsonPath);
+  }

Review comment:
       Or just use a Mockito spy and verify `put` was called, I guess.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705839273



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128);
+  private final int _limit = 1024 * 16;
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {
+      // use putIfAbsent to handle concurrency put

Review comment:
       Both are safe of these two puts, but there will be duplicated put when a cache misses because there is no mutex lock from get to put. I think putIfAbsent can reduce one unnecessary Map node value assignment.It may doesn't matter.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (66df878) into [master](https://codecov.io/gh/apache/pinot/commit/9c2a333f3cc943e350ccbea8c95b804819009e20?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9c2a333) will **decrease** coverage by `41.29%`.
   > The diff coverage is `87.50%`.
   
   > :exclamation: Current head 66df878 differs from pull request most recent head 3166fae. Consider uploading reports for the commit 3166fae to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7409       +/-   ##
   =============================================
   - Coverage     71.98%   30.69%   -41.30%     
   + Complexity     3377       88     -3289     
   =============================================
     Files          1517     1509        -8     
     Lines         75231    74891      -340     
     Branches      10963    10925       -38     
   =============================================
   - Hits          54155    22987    -31168     
   - Misses        17431    49871    +32440     
   + Partials       3645     2033     -1612     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `30.69% <87.50%> (+0.13%)` | :arrow_up: |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `85.71% <85.71%> (ø)` | |
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `84.37% <100.00%> (-2.73%)` | :arrow_down: |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/config/table/TableType.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/DimensionFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EaW1lbnNpb25GaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/FileFormat.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0ZpbGVGb3JtYXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1040 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [9c2a333...3166fae](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514






-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-929855328


   Some tests failed, rerunning to see if intermittent issue due to timeouts.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv removed a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv removed a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930284631


   @Ferrari6 There's unit test failures (I reran a couple of times). Do you mind investigating? cc: @richardstartin 


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e78ac2c) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `40.68%`.
   > The diff coverage is `100.00%`.
   
   > :exclamation: Current head e78ac2c differs from pull request most recent head c03156c. Consider uploading reports for the commit c03156c to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7409       +/-   ##
   =============================================
   - Coverage     71.54%   30.85%   -40.69%     
   =============================================
     Files          1579     1571        -8     
     Lines         80390    80037      -353     
     Branches      11945    11904       -41     
   =============================================
   - Hits          57512    24699    -32813     
   - Misses        18996    53240    +34244     
   + Partials       3882     2098     -1784     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.31% <92.30%> (+0.07%)` | :arrow_up: |
   | integration2 | `27.70% <100.00%> (-0.06%)` | :arrow_down: |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `84.37% <100.00%> (-2.73%)` | :arrow_down: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/config/table/TableType.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/DimensionFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EaW1lbnNpb25GaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/FileFormat.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0ZpbGVGb3JtYXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1061 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...c03156c](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c03156c) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `43.88%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7409       +/-   ##
   =============================================
   - Coverage     71.54%   27.65%   -43.89%     
   =============================================
     Files          1579     1571        -8     
     Lines         80390    80055      -335     
     Branches      11945    11907       -38     
   =============================================
   - Hits          57512    22141    -35371     
   - Misses        18996    55904    +36908     
   + Partials       3882     2010     -1872     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.65% <100.00%> (-0.10%)` | :arrow_down: |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `84.37% <100.00%> (-2.73%)` | :arrow_down: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/config/table/TableType.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/DimensionFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EaW1lbnNpb25GaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/FileFormat.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0ZpbGVGb3JtYXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1125 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...c03156c](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r704592708



##########
File path: pinot-common/src/test/java/org/apache/pinot/common/function/JsonPathMapCacheTest.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.common.function;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Predicate;
+import com.jayway.jsonpath.internal.Utils;
+import com.jayway.jsonpath.spi.cache.Cache;
+import com.jayway.jsonpath.spi.cache.CacheProvider;
+import java.util.Collections;
+import java.util.LinkedList;
+import org.apache.pinot.common.function.scalar.JsonFunctions;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+
+public class JsonPathMapCacheTest {
+
+  private String _jsonString;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    _jsonString = "{\"name\":\"Jack\",\"contact\":{\"phone\":\"01234567\",\"email\":\"test@example.com\"}}";
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheUsed() throws JsonProcessingException {
+    // request json path function, verify SimpleJsonPathMapCache used
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, "$.contact.email"), "test@example.com");
+    Cache cache = CacheProvider.getCache();
+    assertEquals(cache.getClass(), JsonPathMapCache.class);
+  }
+
+  @Test
+  public void testSimpleJsonPathMapCacheWorks() throws JsonProcessingException {
+    String path = "$.contact.email";
+    assertEquals(JsonFunctions.jsonPathString(_jsonString, path), "test@example.com");
+    JsonPathMapCache cache = (JsonPathMapCache) CacheProvider.getCache();
+
+    // verify json path has been cached
+    LinkedList<Predicate> filterStack = new LinkedList<>(Collections.emptyList());
+    String cacheKey = Utils.concat(path, filterStack.toString());
+    JsonPath jsonPath = cache.get(cacheKey);
+    assertNotNull(jsonPath);
+  }

Review comment:
       This depends on undocumented library behaviour, it assumes the format of the cache keys. I have actually submitted a PR to the library to change this behaviour, which would break this test: https://github.com/json-path/JsonPath/pull/750
   
   I suggest iterating over the map instead (requires exposing the keys on `JsonPathMapCache`).




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705954197



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       Whether using Guava's cache would result in high CPU consumption needs measurement. I agree that Caffeine is a better cache implementation, but Guava is already on the classpath. Is this particular use case important enough to add a 1MB depedency?




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e8c7672) into [master](https://codecov.io/gh/apache/pinot/commit/e7901ee8f72c0650dc348e5f9b3cdef7df1566e0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e7901ee) will **decrease** coverage by `9.64%`.
   > The diff coverage is `88.88%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.84%   62.19%   -9.65%     
   + Complexity     3344     3340       -4     
   ============================================
     Files          1517     1509       -8     
     Lines         75039    74700     -339     
     Branches      10921    10883      -38     
   ============================================
   - Hits          53910    46462    -7448     
   - Misses        17504    24877    +7373     
   + Partials       3625     3361     -264     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `28.96% <88.88%> (-0.11%)` | :arrow_down: |
   | unittests1 | `69.70% <88.88%> (+0.01%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/JsonPathMapCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhNYXBDYWNoZS5qYXZh) | `87.50% <87.50%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `81.03% <100.00%> (+0.33%)` | :arrow_up: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/controller/recommender/io/ConfigManager.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9Db25maWdNYW5hZ2VyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...troller/recommender/io/metadata/FieldMetadata.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9tZXRhZGF0YS9GaWVsZE1ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...roller/recommender/rules/impl/BloomFilterRule.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9ydWxlcy9pbXBsL0Jsb29tRmlsdGVyUnVsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ler/recommender/data/generator/BytesGenerator.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9kYXRhL2dlbmVyYXRvci9CeXRlc0dlbmVyYXRvci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...er/recommender/io/metadata/SchemaWithMetaData.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9tZXRhZGF0YS9TY2hlbWFXaXRoTWV0YURhdGEuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...r/recommender/rules/impl/AggregateMetricsRule.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9ydWxlcy9pbXBsL0FnZ3JlZ2F0ZU1ldHJpY3NSdWxlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../recommender/exceptions/InvalidInputException.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9leGNlcHRpb25zL0ludmFsaWRJbnB1dEV4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [268 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e7901ee...e8c7672](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705166716



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128, 0.75f, 1);

Review comment:
       I don't think we need to explicitly set the concurrency level or load factor, these are the default values any way.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r710974036



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       Thanks for the insights @ben-manes. FWIW I have been using Caffeine in commercial projects for years, the only question here is whether _this_ particular use case is worth adding a relatively large (in bytes) dependency.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e8c7672) into [master](https://codecov.io/gh/apache/pinot/commit/e7901ee8f72c0650dc348e5f9b3cdef7df1566e0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e7901ee) will **decrease** coverage by `2.13%`.
   > The diff coverage is `88.88%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.84%   69.70%   -2.14%     
   + Complexity     3344     3255      -89     
   ============================================
     Files          1517     1124     -393     
     Lines         75039    52957   -22082     
     Branches      10921     7970    -2951     
   ============================================
   - Hits          53910    36915   -16995     
   + Misses        17504    13424    -4080     
   + Partials       3625     2618    -1007     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `69.70% <88.88%> (+0.01%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/JsonPathMapCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhNYXBDYWNoZS5qYXZh) | `87.50% <87.50%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `81.03% <100.00%> (+0.33%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...t/core/startree/plan/StarTreeDocIdSetPlanNode.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9wbGFuL1N0YXJUcmVlRG9jSWRTZXRQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [596 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e7901ee...e8c7672](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (fd7aed0) into [master](https://codecov.io/gh/apache/pinot/commit/421645d32f08e1fca2e33af2c20093841ca77098?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (421645d) will **decrease** coverage by `0.02%`.
   > The diff coverage is `85.71%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7409      +/-   ##
   ============================================
   - Coverage     71.54%   71.51%   -0.03%     
   + Complexity     4036     4033       -3     
   ============================================
     Files          1579     1580       +1     
     Lines         80390    80386       -4     
     Branches      11945    11944       -1     
   ============================================
   - Hits          57512    57489      -23     
   - Misses        18996    19012      +16     
   - Partials       3882     3885       +3     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.21% <74.28%> (-0.03%)` | :arrow_down: |
   | integration2 | `27.67% <48.57%> (-0.09%)` | :arrow_down: |
   | unittests1 | `68.61% <80.00%> (-0.05%)` | :arrow_down: |
   | unittests2 | `14.55% <0.00%> (-0.05%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...m/function/JsonExtractScalarTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RTY2FsYXJUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `49.52% <60.00%> (-0.48%)` | :arrow_down: |
   | [...form/function/JsonExtractKeyTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RLZXlUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `78.26% <92.30%> (+4.18%)` | :arrow_up: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `80.35% <100.00%> (-1.00%)` | :arrow_down: |
   | [...a/manager/realtime/RealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVTZWdtZW50RGF0YU1hbmFnZXIuamF2YQ==) | `50.00% <0.00%> (-25.00%)` | :arrow_down: |
   | [...nt/local/startree/v2/store/StarTreeDataSource.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zdGFydHJlZS92Mi9zdG9yZS9TdGFyVHJlZURhdGFTb3VyY2UuamF2YQ==) | `40.00% <0.00%> (-13.34%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/common/utils/URIUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvVVJJVXRpbHMuamF2YQ==) | `66.66% <0.00%> (-7.41%)` | :arrow_down: |
   | [.../common/request/context/predicate/EqPredicate.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVxdWVzdC9jb250ZXh0L3ByZWRpY2F0ZS9FcVByZWRpY2F0ZS5qYXZh) | `66.66% <0.00%> (-6.67%)` | :arrow_down: |
   | [...mmon/request/context/predicate/NotEqPredicate.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVxdWVzdC9jb250ZXh0L3ByZWRpY2F0ZS9Ob3RFcVByZWRpY2F0ZS5qYXZh) | `66.66% <0.00%> (-6.67%)` | :arrow_down: |
   | [...elix/core/periodictask/ControllerPeriodicTask.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3BlcmlvZGljdGFzay9Db250cm9sbGVyUGVyaW9kaWNUYXNrLmphdmE=) | `76.00% <0.00%> (-6.00%)` | :arrow_down: |
   | ... and [21 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [421645d...fd7aed0](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] ben-manes commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r710774717



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.

Review comment:
       The performance of cache reads improves with the concurrency level due to reducing the likelihood of contending on the same CLQ instance. The benchmarks use a zipf to simulate a hotspot.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] richardstartin commented on a change in pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#discussion_r705165725



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/JsonPathMapCache.java
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.pinot.common.function;
+
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.cache.Cache;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This cache for avoiding repeatedly compiling JSON path string when using jsonPathXXX transformations
+ *
+ * <p>The existing jay way LRUCache is very inefficient, It will cause
+ * a lot of unnecessary lock waits during high concurrent data ingestion,
+ * and LRU mechanism is inappropriate for Pinot bounded size of the
+ * transformation config, so we should use this simple Map cache instead
+ * of it.
+ */
+public class JsonPathMapCache implements Cache {
+  private final ConcurrentMap<String, JsonPath> _pathCache = new ConcurrentHashMap<>(128, 0.75f, 1);
+
+  private final int _limit;
+
+  public JsonPathMapCache(int limit) {
+    this._limit = limit;
+  }
+
+  @Override
+  public JsonPath get(String key) {
+    return _pathCache.get(key);
+  }
+
+  /**
+   * cache JsonPath, if reached size limit, not cache it anymore.
+   */
+  @Override
+  public void put(String key, JsonPath value) {
+    if (this.size() < _limit) {

Review comment:
       This is obviously racy and doesn't provide a hard bound, but I don't think it matters.




-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] snleee edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
snleee edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-920374117


   @Ferrari6 Can you rebase based on the master branch and retrigger github action tests? We recently introduced the flakyness to one of our integration test and fixed from https://github.com/apache/pinot/pull/7432


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Ferrari6 commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
Ferrari6 commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-920437691


   > @Ferrari6 Can you rebase based on the master branch and retrigger github action tests? We recently introduced the flakyness to one of our integration test and fixed from #7432
   
   yes, sure. But because the setCache of CacheProvider in Jayway can only be called once, and once getCache is called, it cannot be setCache again. So I need to do some code changes.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] mayankshriv commented on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-930284631


   @Ferrari6 There's unit test failures (I reran a couple of times). Do you mind investigating? cc: @richardstartin 


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #7409: Fix JSONPath cache inefficient issue

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7409:
URL: https://github.com/apache/pinot/pull/7409#issuecomment-915143514


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7409](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2be13d1) into [master](https://codecov.io/gh/apache/pinot/commit/240bcb80546454d5ab449d3d3b13867f4943706d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (240bcb8) will **decrease** coverage by `0.00%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7409/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7409      +/-   ##
   ==========================================
   - Coverage   32.19%   32.19%   -0.01%     
   ==========================================
     Files        1514     1515       +1     
     Lines       75167    75176       +9     
     Branches    10966    10966              
   ==========================================
   - Hits        24203    24202       -1     
   - Misses      48870    48891      +21     
   + Partials     2094     2083      -11     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `30.38% <88.88%> (-0.02%)` | :arrow_down: |
   | integration2 | `29.13% <100.00%> (+0.11%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `84.37% <100.00%> (+0.50%)` | :arrow_up: |
   | [...rg/apache/pinot/common/function/JsonPathCache.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vSnNvblBhdGhDYWNoZS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...a/org/apache/pinot/common/utils/DateTimeUtils.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvRGF0ZVRpbWVVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/restlet/resources/SegmentErrorInfo.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVzdGxldC9yZXNvdXJjZXMvU2VnbWVudEVycm9ySW5mby5qYXZh) | `0.00% <0.00%> (-30.77%)` | :arrow_down: |
   | [...a/manager/realtime/RealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVTZWdtZW50RGF0YU1hbmFnZXIuamF2YQ==) | `50.00% <0.00%> (-25.00%)` | :arrow_down: |
   | [...ion/groupby/AggregationGroupByTrimmingService.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9ncm91cGJ5L0FnZ3JlZ2F0aW9uR3JvdXBCeVRyaW1taW5nU2VydmljZS5qYXZh) | `66.66% <0.00%> (-23.92%)` | :arrow_down: |
   | [...ller/helix/core/minion/TaskTypeMetricsUpdater.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrVHlwZU1ldHJpY3NVcGRhdGVyLmphdmE=) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [.../filter/predicate/InPredicateEvaluatorFactory.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvcHJlZGljYXRlL0luUHJlZGljYXRlRXZhbHVhdG9yRmFjdG9yeS5qYXZh) | `50.50% <0.00%> (-10.11%)` | :arrow_down: |
   | [...nction/DistinctCountBitmapAggregationFunction.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9EaXN0aW5jdENvdW50Qml0bWFwQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `39.69% <0.00%> (-4.13%)` | :arrow_down: |
   | [...rg/apache/pinot/core/query/utils/idset/IdSets.java](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS91dGlscy9pZHNldC9JZFNldHMuamF2YQ==) | `25.71% <0.00%> (-2.86%)` | :arrow_down: |
   | ... and [17 more](https://codecov.io/gh/apache/pinot/pull/7409/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [240bcb8...2be13d1](https://codecov.io/gh/apache/pinot/pull/7409?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org