You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "kirkrodrigues (via GitHub)" <gi...@apache.org> on 2023/06/29 23:29:28 UTC

[GitHub] [pinot] kirkrodrigues opened a new pull request, #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

kirkrodrigues opened a new pull request, #11006:
URL: https://github.com/apache/pinot/pull/11006

   tags: feature
   
   This adds a query rewriter to make it easier to call `clpDecode`. Recall that the [`CLPLogMessageDecoder`](https://github.com/apache/pinot/pull/9942) encodes unstructured log fields into three columns, each with a common prefix (and different suffixes). We refer to this common prefix as the column-group name. This PR adds support for calling `clpDecode` using the column-group name rather than specifying the three individual columns.
   
   E.g., if the `message` field was encoded with CLP, users currently have to call `clpDecode` as follows to reconstruct the field's original value:
   
   `clpDecode(message_logtype, message_dictionaryVars, message_encodedVars)`
   
   After this PR, users can call `clpDecode` as follows:
   
   `clpDecode(message)`
   
   To use the rewriter, users need to change their broker config to add `org.apache.pinot.sql.parsers.rewriter.CLPDecodeRewriter` to `pinot.broker.query.rewriter.class.names`. Assuming the default set of query rewriters, that would look like:
   
   ```
   org.apache.pinot.sql.parsers.rewriter.CompileTimeFunctionsInvoker,org.apache.pinot.sql.parsers.rewriter.SelectionsRewriter,org.apache.pinot.sql.parsers.rewriter.PredicateComparisonRewriter,org.apache.pinot.sql.parsers.rewriter.OrdinalsUpdater,org.apache.pinot.sql.parsers.rewriter.CLPDecodeRewriter,org.apache.pinot.sql.parsers.rewriter.AliasApplier,org.apache.pinot.sql.parsers.rewriter.NonAggregationGroupByToDistinctQueryRewriter
   ```
   Note that we added it before the `AliasApplier` so that any aliasing of `message` happens only after the `clpDecode` rewrite.
   
   This is part of the change requested in #9819 and described in this [design doc](https://docs.google.com/document/d/1nHZb37re4mUwEA258x3a2pgX13EWLWMJ0uLEDk1dUyU/edit#heading=h.x12tsj9ok16d).
   
   Note also that this is a precursor to `clpMatch` which will be a much more involved query rewriter.
   
   # Testing performed
   * Added new unit tests.
   * Validated fields encoded with CLP could be decoded correctly, using the column-group name.
   


-- 
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] chenboat merged pull request #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat merged PR #11006:
URL: https://github.com/apache/pinot/pull/11006


-- 
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] chenboat commented on a diff in pull request #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #11006:
URL: https://github.com/apache/pinot/pull/11006#discussion_r1248338626


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/CLPDecodeRewriter.java:
##########
@@ -0,0 +1,176 @@
+/**
+ * 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.sql.parsers.rewriter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.function.TransformFunctionType;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Identifier;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.request.PinotQuery;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+
+
+/**
+ * Query rewriter to rewrite clpDecode so that users can pass in the name of a CLP-encoded column group instead of the
+ * names of all the columns in the group.
+ * <p>
+ * Usage:
+ * <pre>
+ *   clpDecode("columnGroupName"[, defaultValue])
+ * </pre>
+ * which will be rewritten to:
+ * <pre>
+ *   clpDecode("columnGroupName_logtype", "columnGroupName_dictionaryVars", "columnGroupName_encodedVars"[,
+ *   defaultValue])
+ * </pre>
+ * The "defaultValue" is optional. See
+ * {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for its description.
+ * <p>
+ * Sample queries:
+ * <pre>
+ *   SELECT clpDecode("message") FROM table
+ *   SELECT clpDecode("message", 'null') FROM table
+ * </pre>
+ * See {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for details about the
+ * underlying clpDecode transformer.
+ */
+public class CLPDecodeRewriter implements QueryRewriter {
+  public static final String LOGTYPE_COLUMN_SUFFIX = "_logtype";
+  public static final String DICTIONARY_VARS_COLUMN_SUFFIX = "_dictionaryVars";
+  public static final String ENCODED_VARS_COLUMN_SUFFIX = "_encodedVars";
+
+  private static final String _CLPDECODE_LOWERCASE_TRANSFORM_NAME =
+      TransformFunctionType.CLPDECODE.getName().toLowerCase();
+
+  @Override
+  public PinotQuery rewrite(PinotQuery pinotQuery) {
+    List<Expression> selectExpressions = pinotQuery.getSelectList();
+    if (null != selectExpressions) {
+      for (Expression e : selectExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> groupByExpressions = pinotQuery.getGroupByList();
+    if (null != groupByExpressions) {
+      for (Expression e : groupByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> orderByExpressions = pinotQuery.getOrderByList();
+    if (null != orderByExpressions) {
+      for (Expression e : orderByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    tryRewritingExpression(pinotQuery.getFilterExpression());
+    tryRewritingExpression(pinotQuery.getHavingExpression());
+    return pinotQuery;
+  }
+
+  /**
+   * Rewrites any instances of clpDecode in the given expression
+   * @param expression Expression which may contain instances of clpDecode
+   */
+  private void tryRewritingExpression(Expression expression) {
+    if (null == expression) {
+      return;
+    }
+    Function function = expression.getFunctionCall();
+    if (null == function) {
+      return;
+    }
+
+    String functionName = function.getOperator();
+    if (functionName.equals(_CLPDECODE_LOWERCASE_TRANSFORM_NAME)) {
+      rewriteCLPDecodeFunction(expression);
+    } else {
+      // Function isn't a CLP function that needs rewriting, but the arguments might be, so we recursively process them.
+      for (Expression op : function.getOperands()) {
+        tryRewritingExpression(op);
+      }
+    }
+  }
+
+  /**
+   * Rewrites the given instance of clpDecode as described in the class docstring

Review Comment:
   what is docstring? typo?



-- 
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] kirkrodrigues commented on a diff in pull request #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

Posted by "kirkrodrigues (via GitHub)" <gi...@apache.org>.
kirkrodrigues commented on code in PR #11006:
URL: https://github.com/apache/pinot/pull/11006#discussion_r1248344244


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/CLPDecodeRewriter.java:
##########
@@ -0,0 +1,176 @@
+/**
+ * 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.sql.parsers.rewriter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.function.TransformFunctionType;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Identifier;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.request.PinotQuery;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+
+
+/**
+ * Query rewriter to rewrite clpDecode so that users can pass in the name of a CLP-encoded column group instead of the
+ * names of all the columns in the group.
+ * <p>
+ * Usage:
+ * <pre>
+ *   clpDecode("columnGroupName"[, defaultValue])
+ * </pre>
+ * which will be rewritten to:
+ * <pre>
+ *   clpDecode("columnGroupName_logtype", "columnGroupName_dictionaryVars", "columnGroupName_encodedVars"[,
+ *   defaultValue])
+ * </pre>
+ * The "defaultValue" is optional. See
+ * {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for its description.
+ * <p>
+ * Sample queries:
+ * <pre>
+ *   SELECT clpDecode("message") FROM table
+ *   SELECT clpDecode("message", 'null') FROM table
+ * </pre>
+ * See {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for details about the
+ * underlying clpDecode transformer.
+ */
+public class CLPDecodeRewriter implements QueryRewriter {
+  public static final String LOGTYPE_COLUMN_SUFFIX = "_logtype";
+  public static final String DICTIONARY_VARS_COLUMN_SUFFIX = "_dictionaryVars";
+  public static final String ENCODED_VARS_COLUMN_SUFFIX = "_encodedVars";
+
+  private static final String _CLPDECODE_LOWERCASE_TRANSFORM_NAME =
+      TransformFunctionType.CLPDECODE.getName().toLowerCase();
+
+  @Override
+  public PinotQuery rewrite(PinotQuery pinotQuery) {
+    List<Expression> selectExpressions = pinotQuery.getSelectList();
+    if (null != selectExpressions) {
+      for (Expression e : selectExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> groupByExpressions = pinotQuery.getGroupByList();
+    if (null != groupByExpressions) {
+      for (Expression e : groupByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> orderByExpressions = pinotQuery.getOrderByList();
+    if (null != orderByExpressions) {
+      for (Expression e : orderByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    tryRewritingExpression(pinotQuery.getFilterExpression());
+    tryRewritingExpression(pinotQuery.getHavingExpression());
+    return pinotQuery;
+  }
+
+  /**
+   * Rewrites any instances of clpDecode in the given expression
+   * @param expression Expression which may contain instances of clpDecode
+   */
+  private void tryRewritingExpression(Expression expression) {
+    if (null == expression) {
+      return;
+    }
+    Function function = expression.getFunctionCall();
+    if (null == function) {
+      return;
+    }
+
+    String functionName = function.getOperator();
+    if (functionName.equals(_CLPDECODE_LOWERCASE_TRANSFORM_NAME)) {
+      rewriteCLPDecodeFunction(expression);
+    } else {
+      // Function isn't a CLP function that needs rewriting, but the arguments might be, so we recursively process them.
+      for (Expression op : function.getOperands()) {
+        tryRewritingExpression(op);
+      }
+    }
+  }
+
+  /**
+   * Rewrites the given instance of clpDecode as described in the class docstring

Review Comment:
   Ah, I think it's a [general term](https://en.wikipedia.org/wiki/Docstring) for API documentation comments, but the more specific term in Java is a Javadoc, so switched to 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.

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 #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #11006:
URL: https://github.com/apache/pinot/pull/11006#issuecomment-1613942683

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11006?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11006](https://app.codecov.io/gh/apache/pinot/pull/11006?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (c8fb759) into [master](https://app.codecov.io/gh/apache/pinot/commit/9374964ad4b5797329333157fcbb1de56cc66657?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (9374964) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #11006     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2191     2138     -53     
     Lines      118007   115567   -2440     
     Branches    17868    17580    -288     
   =========================================
     Hits          137      137             
   + Misses     117850   115410   -2440     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests2temurin20 | `?` | |
   
   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=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/11006?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [.../pinot/sql/parsers/rewriter/CLPDecodeRewriter.java](https://app.codecov.io/gh/apache/pinot/pull/11006?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9yZXdyaXRlci9DTFBEZWNvZGVSZXdyaXRlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   
   ... and [56 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11006/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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] chenboat commented on a diff in pull request #11006: Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns.

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #11006:
URL: https://github.com/apache/pinot/pull/11006#discussion_r1248338357


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/CLPDecodeRewriter.java:
##########
@@ -0,0 +1,176 @@
+/**
+ * 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.sql.parsers.rewriter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.function.TransformFunctionType;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Identifier;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.request.PinotQuery;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+
+
+/**
+ * Query rewriter to rewrite clpDecode so that users can pass in the name of a CLP-encoded column group instead of the
+ * names of all the columns in the group.
+ * <p>
+ * Usage:
+ * <pre>
+ *   clpDecode("columnGroupName"[, defaultValue])
+ * </pre>
+ * which will be rewritten to:
+ * <pre>
+ *   clpDecode("columnGroupName_logtype", "columnGroupName_dictionaryVars", "columnGroupName_encodedVars"[,
+ *   defaultValue])
+ * </pre>
+ * The "defaultValue" is optional. See
+ * {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for its description.
+ * <p>
+ * Sample queries:
+ * <pre>
+ *   SELECT clpDecode("message") FROM table
+ *   SELECT clpDecode("message", 'null') FROM table
+ * </pre>
+ * See {@link org.apache.pinot.core.operator.transform.function.CLPDecodeTransformFunction} for details about the
+ * underlying clpDecode transformer.
+ */
+public class CLPDecodeRewriter implements QueryRewriter {
+  public static final String LOGTYPE_COLUMN_SUFFIX = "_logtype";
+  public static final String DICTIONARY_VARS_COLUMN_SUFFIX = "_dictionaryVars";
+  public static final String ENCODED_VARS_COLUMN_SUFFIX = "_encodedVars";
+
+  private static final String _CLPDECODE_LOWERCASE_TRANSFORM_NAME =
+      TransformFunctionType.CLPDECODE.getName().toLowerCase();
+
+  @Override
+  public PinotQuery rewrite(PinotQuery pinotQuery) {
+    List<Expression> selectExpressions = pinotQuery.getSelectList();
+    if (null != selectExpressions) {
+      for (Expression e : selectExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> groupByExpressions = pinotQuery.getGroupByList();
+    if (null != groupByExpressions) {
+      for (Expression e : groupByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    List<Expression> orderByExpressions = pinotQuery.getOrderByList();
+    if (null != orderByExpressions) {
+      for (Expression e : orderByExpressions) {
+        tryRewritingExpression(e);
+      }
+    }
+    tryRewritingExpression(pinotQuery.getFilterExpression());
+    tryRewritingExpression(pinotQuery.getHavingExpression());
+    return pinotQuery;
+  }
+
+  /**
+   * Rewrites any instances of clpDecode in the given expression
+   * @param expression Expression which may contain instances of clpDecode
+   */
+  private void tryRewritingExpression(Expression expression) {
+    if (null == expression) {
+      return;
+    }
+    Function function = expression.getFunctionCall();
+    if (null == function) {
+      return;
+    }
+
+    String functionName = function.getOperator();
+    if (functionName.equals(_CLPDECODE_LOWERCASE_TRANSFORM_NAME)) {
+      rewriteCLPDecodeFunction(expression);

Review Comment:
   nit: return here and save the else branch.



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