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/01/27 00:36:21 UTC

[GitHub] [incubator-pinot] Jackie-Jiang opened a new pull request #6494: Support generating derived column during segment load

Jackie-Jiang opened a new pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494


   ## Description
   Currently we allow transform function in the ingestion config to generate derived columns during segment creation. This PR adds the support for generating the derived columns during segment load so that derived columns can be added on-the-fly.
   
   TODOs:
    - Support chained derived column
    - Support raw derived column
    - Support multi-value derived column


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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {

Review comment:
       We only want to make this check for derived column. For default value column, we don't need this check and always generate dictionary-encoded indices for better compression




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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);

Review comment:
       After going over the code again, the code change is not much. Applied the logic within this PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -414,4 +496,186 @@ protected void createColumnV1Indices(String column)
             true/*hasDictionary*/, dictionaryElementSize, true/*hasInvertedIndex*/, TextIndexType.NONE,
             false/*hasFSTIndex*/, false/*hasJsonIndex*/);
   }
+
+  /**
+   * Helper method to create the V1 indices (dictionary and forward index) for a column with derived values.
+   * TODO:
+   *   - Support chained derived column
+   *   - Support raw derived column
+   *   - Support multi-value derived column
+   */
+  private void createDerivedColumnV1Indices(String column, FunctionEvaluator functionEvaluator,
+      List<ColumnMetadata> argumentsMetadata)
+      throws Exception {
+    // Initialize value readers for all arguments
+    int numArguments = argumentsMetadata.size();
+    List<ValueReader> valueReaders = new ArrayList<>(numArguments);
+    for (ColumnMetadata argumentMetadata : argumentsMetadata) {
+      valueReaders.add(new ValueReader(argumentMetadata));
+    }
+
+    try {
+      // Calculate the values for the derived column
+      Object[] inputValues = new Object[numArguments];
+      int numDocs = _segmentMetadata.getTotalDocs();
+      Object[] outputValues = new Object[numDocs];
+      for (int i = 0; i < numDocs; i++) {
+        for (int j = 0; j < numArguments; j++) {
+          inputValues[j] = valueReaders.get(j).getValue(i);
+        }
+        outputValues[i] = functionEvaluator.evaluate(inputValues);
+      }
+
+      FieldSpec fieldSpec = _schema.getFieldSpecFor(column);
+      StatsCollectorConfig statsCollectorConfig =
+          new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+      ColumnIndexCreationInfo indexCreationInfo;
+      switch (fieldSpec.getDataType()) {

Review comment:
       I don't think separating them into separate functions is more readable because it is not reused anywhere else, and keeping them together is more concise and easier to read IMO.




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

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566526224



##########
File path: pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
##########
@@ -53,6 +53,8 @@
 import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.config.table.ingestion.TransformConfig;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.JsonUtils;

Review comment:
       In the past, we have added tests to SegmentPreprocessorTest.java as well May be add one there to cover derived columns?




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

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] [incubator-pinot] fx19880617 commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
fx19880617 commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r564951218



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          // TODO: Support multi-value derived column
+          if (!_schema.getFieldSpecFor(column).isSingleValueField()) {
+            LOGGER.warn("Skip creating MV derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          try {
+            createDerivedColumnV1Indices(column, functionEvaluator, argumentsMetadata);
+            return;
+          } catch (Exception e) {
+            LOGGER.error(

Review comment:
       If the user misconfigured the transform function and found out the values are all default values, is there a way to fix this?
   E.g. I can force re-generate the derived column.




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

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566529435



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {

Review comment:
       We can use break outer in the inner loop (by labelling the outer for loop) and avoid this if block




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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);

Review comment:
       I've considered that, but that requires more code change because we assume the default column is always added, and the metadata will be changed accordingly.
   There is a work-around to re-download the segments if the user puts the unsupported config, they can update the schema with a different default null value, which will trigger the update of the default column.




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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
##########
@@ -53,6 +53,8 @@
 import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.config.table.ingestion.TransformConfig;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.JsonUtils;

Review comment:
       Good suggestion, added.




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

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] [incubator-pinot] Jackie-Jiang merged pull request #6494: Support generating derived column during segment load

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


   


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

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] [incubator-pinot] codecov-io commented on pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#issuecomment-769539620


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=h1) Report
   > Merging [#6494](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=desc) (5d8831b) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.72%`.
   > The diff coverage is `59.96%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6494/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6494      +/-   ##
   ==========================================
   - Coverage   66.44%   64.72%   -1.73%     
   ==========================================
     Files        1075     1334     +259     
     Lines       54773    65786   +11013     
     Branches     8168     9605    +1437     
   ==========================================
   + Hits        36396    42580    +6184     
   - Misses      15700    20159    +4459     
   - Partials     2677     3047     +370     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `64.72% <59.96%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <ø> (+9.52%)` | :arrow_up: |
   | [...n/java/org/apache/pinot/client/BrokerResponse.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Jyb2tlclJlc3BvbnNlLmphdmE=) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <ø> (-13.29%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/client/ExecutionStats.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0V4ZWN1dGlvblN0YXRzLmphdmE=) | `15.55% <ø> (ø)` | |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <ø> (-51.10%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/client/ResultSetGroup.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L1Jlc3VsdFNldEdyb3VwLmphdmE=) | `65.38% <ø> (+0.16%)` | :arrow_up: |
   | ... and [1184 more](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=footer). Last update [bacaed2...5d8831b](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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] [incubator-pinot] fx19880617 commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
fx19880617 commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r565901781



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          // TODO: Support multi-value derived column
+          if (!_schema.getFieldSpecFor(column).isSingleValueField()) {
+            LOGGER.warn("Skip creating MV derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          try {
+            createDerivedColumnV1Indices(column, functionEvaluator, argumentsMetadata);
+            return;
+          } catch (Exception e) {
+            LOGGER.error(

Review comment:
       Make sense, maybe we should provide an API to force download the segment then reload it.




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

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566524925



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -414,4 +496,186 @@ protected void createColumnV1Indices(String column)
             true/*hasDictionary*/, dictionaryElementSize, true/*hasInvertedIndex*/, TextIndexType.NONE,
             false/*hasFSTIndex*/, false/*hasJsonIndex*/);
   }
+
+  /**
+   * Helper method to create the V1 indices (dictionary and forward index) for a column with derived values.
+   * TODO:
+   *   - Support chained derived column
+   *   - Support raw derived column
+   *   - Support multi-value derived column
+   */
+  private void createDerivedColumnV1Indices(String column, FunctionEvaluator functionEvaluator,
+      List<ColumnMetadata> argumentsMetadata)
+      throws Exception {
+    // Initialize value readers for all arguments
+    int numArguments = argumentsMetadata.size();
+    List<ValueReader> valueReaders = new ArrayList<>(numArguments);
+    for (ColumnMetadata argumentMetadata : argumentsMetadata) {
+      valueReaders.add(new ValueReader(argumentMetadata));
+    }
+
+    try {
+      // Calculate the values for the derived column
+      Object[] inputValues = new Object[numArguments];
+      int numDocs = _segmentMetadata.getTotalDocs();
+      Object[] outputValues = new Object[numDocs];
+      for (int i = 0; i < numDocs; i++) {
+        for (int j = 0; j < numArguments; j++) {
+          inputValues[j] = valueReaders.get(j).getValue(i);
+        }
+        outputValues[i] = functionEvaluator.evaluate(inputValues);
+      }
+
+      FieldSpec fieldSpec = _schema.getFieldSpecFor(column);
+      StatsCollectorConfig statsCollectorConfig =
+          new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+      ColumnIndexCreationInfo indexCreationInfo;
+      switch (fieldSpec.getDataType()) {

Review comment:
       I suggest to separate the switch block into separate function if possible




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

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] [incubator-pinot] npawar commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566420274



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);

Review comment:
       In both these cases, should we just skip creating anything, instead of creating defaultColumn? User was not aware that this is not allowed. We should give them a chance to go put a defaultNullValue of their choice. It is not easy to undo this default column materialization - reingesting can be cumbersome.




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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {

Review comment:
       I think label break is bad practice in java? I've replaced it with return and removed the skip flag to apply the change @npawar suggested




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

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566529207



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {

Review comment:
       We can make this check before the if at line 349 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.

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566526224



##########
File path: pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
##########
@@ -53,6 +53,8 @@
 import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.config.table.ingestion.TransformConfig;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.JsonUtils;

Review comment:
       In the past, we have added tests to SegmentPreprocessorTest.java as well. May be add one there to cover derived columns as well?




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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          // TODO: Support multi-value derived column
+          if (!_schema.getFieldSpecFor(column).isSingleValueField()) {
+            LOGGER.warn("Skip creating MV derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          try {
+            createDerivedColumnV1Indices(column, functionEvaluator, argumentsMetadata);
+            return;
+          } catch (Exception e) {
+            LOGGER.error(

Review comment:
       Fixed this by skipping creating derived column if not possible instead of creating default value column as Neha suggested




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

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] [incubator-pinot] siddharthteotia commented on a change in pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#discussion_r566526224



##########
File path: pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
##########
@@ -53,6 +53,8 @@
 import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.config.table.ingestion.TransformConfig;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.JsonUtils;

Review comment:
       In the past, we have added tests to SegmentPreprocessorTest.java as well. May be add one there as well to cover derived columns as well?




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

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] [incubator-pinot] codecov-io edited a comment on pull request #6494: Support generating derived column during segment load

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6494:
URL: https://github.com/apache/incubator-pinot/pull/6494#issuecomment-769539620


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=h1) Report
   > Merging [#6494](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=desc) (8a032b6) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `22.45%`.
   > The diff coverage is `42.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6494/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #6494       +/-   ##
   ===========================================
   - Coverage   66.44%   43.99%   -22.46%     
   ===========================================
     Files        1075     1334      +259     
     Lines       54773    65788    +11015     
     Branches     8168     9605     +1437     
   ===========================================
   - Hits        36396    28944     -7452     
   - Misses      15700    34399    +18699     
   + Partials     2677     2445      -232     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `43.99% <42.05%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...org/apache/pinot/broker/queryquota/HitCounter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9IaXRDb3VudGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/broker/queryquota/MaxHitRateTracker.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9NYXhIaXRSYXRlVHJhY2tlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ache/pinot/broker/queryquota/QueryQuotaEntity.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9RdWVyeVF1b3RhRW50aXR5LmphdmE=) | `0.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ceselector/StrictReplicaGroupInstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL1N0cmljdFJlcGxpY2FHcm91cEluc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...roker/routing/segmentpruner/TimeSegmentPruner.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1RpbWVTZWdtZW50UHJ1bmVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...roker/routing/segmentpruner/interval/Interval.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL2ludGVydmFsL0ludGVydmFsLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...r/routing/segmentpruner/interval/IntervalTree.java](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL2ludGVydmFsL0ludGVydmFsVHJlZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [1331 more](https://codecov.io/gh/apache/incubator-pinot/pull/6494/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=footer). Last update [bacaed2...8a032b6](https://codecov.io/gh/apache/incubator-pinot/pull/6494?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6494: Support generating derived column during segment load

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -324,8 +345,69 @@ protected void removeColumnV1Indices(String column)
    */
   protected void createColumnV1Indices(String column)
       throws Exception {
+    TableConfig tableConfig = _indexLoadingConfig.getTableConfig();
+    if (tableConfig != null && tableConfig.getIngestionConfig() != null
+        && tableConfig.getIngestionConfig().getTransformConfigs() != null) {
+      List<TransformConfig> transformConfigs = tableConfig.getIngestionConfig().getTransformConfigs();
+      for (TransformConfig transformConfig : transformConfigs) {
+        if (transformConfig.getColumnName().equals(column)) {
+          String transformFunction = transformConfig.getTransformFunction();
+          FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformFunction);
+
+          // Check if all arguments exist in the segment
+          // TODO: Support chained derived column
+          boolean skipCreatingDerivedColumn = false;
+          List<String> arguments = functionEvaluator.getArguments();
+          List<ColumnMetadata> argumentsMetadata = new ArrayList<>(arguments.size());
+          for (String argument : arguments) {
+            ColumnMetadata columnMetadata = _segmentMetadata.getColumnMetadataFor(argument);
+            if (columnMetadata == null) {
+              LOGGER.warn(
+                  "Skip creating derived column: {} because argument: {} does not exist in the segment, creating default value column instead",
+                  column, argument);
+              skipCreatingDerivedColumn = true;
+              break;
+            }
+            argumentsMetadata.add(columnMetadata);
+          }
+          if (skipCreatingDerivedColumn) {
+            break;
+          }
+
+          // TODO: Support raw derived column
+          if (_indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
+            LOGGER.warn("Skip creating raw derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          // TODO: Support multi-value derived column
+          if (!_schema.getFieldSpecFor(column).isSingleValueField()) {
+            LOGGER.warn("Skip creating MV derived column: {}, creating default value column instead", column);
+            break;
+          }
+
+          try {
+            createDerivedColumnV1Indices(column, functionEvaluator, argumentsMetadata);
+            return;
+          } catch (Exception e) {
+            LOGGER.error(

Review comment:
       Hmm, good question. Currently Pinot does not support removing/updating existing index in v3 format, so the only way to re-generate the column is to download a new segment from the controller. Once we have index modification supported, we can add that




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

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