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/30 23:43:40 UTC

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

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