You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/09/10 05:31:18 UTC

[GitHub] [druid] gianm commented on a change in pull request #10350: Support SearchQueryDimFilter in sql via new methods

gianm commented on a change in pull request #10350:
URL: https://github.com/apache/druid/pull/10350#discussion_r486068975



##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ContainsOperatorConversion.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.druid.sql.calcite.expression.builtin;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.filter.SearchQueryDimFilter;
+import org.apache.druid.query.search.ContainsSearchQuerySpec;
+import org.apache.druid.query.search.SearchQuerySpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.expression.Expressions;
+import org.apache.druid.sql.calcite.expression.OperatorConversions;
+import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
+import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+public class ContainsOperatorConversion extends DirectOperatorConversion
+{
+  private static final String CASE_SENSITIVE_FN_NAME = "contains_str";
+  private static final String CASE_INSENSITIVE_FN_NAME = "icontains_str";
+  private final boolean caseSensitive;
+
+  public ContainsOperatorConversion(
+      final SqlFunction sqlFunction,
+      final String functionName,
+      final boolean caseSensitive
+  )
+  {
+    super(sqlFunction, functionName);

Review comment:
       I don't see native functions named "contains_str" and "icontains_str", do they exist? (DirectOperatorConversion assumes there is a native function with the functionName you pass in here.)

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java
##########
@@ -1072,6 +1075,108 @@ public void testPad()
     );
   }
 
+  @Test
+  public void testContains()
+  {

Review comment:
       You should include tests here using `testHelper.testExpression` too. It'll test that `toDruidExpression` works properly, which will be used if someone uses this operator outside of a leaf filter.

##########
File path: docs/querying/sql.md
##########
@@ -561,6 +561,8 @@ The [DataSketches extension](../development/extensions-core/datasketches-extensi
 |`COALESCE(value1, value2, ...)`|Returns the first value that is neither NULL nor empty string.|
 |`NVL(expr,expr-for-null)`|Returns 'expr-for-null' if 'expr' is null (or empty string for string type).|
 |`BLOOM_FILTER_TEST(<expr>, <serialized-filter>)`|Returns true if the value is contained in a Base64-serialized bloom filter. See the [Bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
+|`CONTAINS_STR(<expr>, str)`|Returns true if the `str` is a substring of `expr`.|
+|`ICONTAINS_STR(<expr>, str)`|Returns true if the `str` is a substring of `expr`. The match is case-insensitive.|

Review comment:
       These should be in the "String functions" section.
   
   I don't have an opinion on the names right now, but to form an opinion I like to do the following.
   
   1. Check Calcite's SqlStdOperatorTable, which contains standard operators. If there's something matching there, we should use that.
   2. If there isn't a standard operator, survey some other databases. I like to check the golden oldies: MySQL, PostgreSQL, Oracle, SQL Server, as well as some of the newer ones like Presto, Snowflake, BigQuery. If a few of them seem to agree on a name and behavior it's good to go with that.
   3. If there doesn't seem to be any agreement, or if you don't want to implement what they seem to agree on for some reason, then make up a new operator.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java
##########
@@ -181,6 +182,8 @@
           .add(new AliasedOperatorConversion(new TruncateOperatorConversion(), "TRUNC"))
           .add(new LPadOperatorConversion())
           .add(new RPadOperatorConversion())
+          .add(ContainsOperatorConversion.createOperatorConversion(true))
+          .add(ContainsOperatorConversion.createOperatorConversion(false))

Review comment:
       Might be more readable to give these methods nice names, like `caseInsensitive()` and `caseSensitive()`. Small nit though.




----------------------------------------------------------------
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org