mirror of https://github.com/apache/druid.git
SQL: Add "POSITION" function. (#6596)
Also add a "fromIndex" argument to the strpos expression function. There are some -1 and +1 adjustment terms due to the fact that the strpos expression behaves like Java indexOf (0-indexed), but the POSITION SQL function is 1-indexed.
This commit is contained in:
parent
f2b73f9df1
commit
154b6fbcef
|
@ -955,8 +955,8 @@ interface Function
|
||||||
@Override
|
@Override
|
||||||
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
|
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
if (args.size() != 2) {
|
if (args.size() < 2 || args.size() > 3) {
|
||||||
throw new IAE("Function[%s] needs 2 arguments", name());
|
throw new IAE("Function[%s] needs 2 or 3 arguments", name());
|
||||||
}
|
}
|
||||||
|
|
||||||
final String haystack = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString());
|
final String haystack = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString());
|
||||||
|
@ -965,7 +965,16 @@ interface Function
|
||||||
if (haystack == null || needle == null) {
|
if (haystack == null || needle == null) {
|
||||||
return ExprEval.of(null);
|
return ExprEval.of(null);
|
||||||
}
|
}
|
||||||
return ExprEval.of(haystack.indexOf(needle));
|
|
||||||
|
final int fromIndex;
|
||||||
|
|
||||||
|
if (args.size() >= 3) {
|
||||||
|
fromIndex = args.get(2).eval(bindings).asInt();
|
||||||
|
} else {
|
||||||
|
fromIndex = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
return ExprEval.of(haystack.indexOf(needle, fromIndex));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -95,6 +95,10 @@ public class FunctionTest
|
||||||
public void testStrpos()
|
public void testStrpos()
|
||||||
{
|
{
|
||||||
assertExpr("strpos(x, 'o')", 1L);
|
assertExpr("strpos(x, 'o')", 1L);
|
||||||
|
assertExpr("strpos(x, 'o', 0)", 1L);
|
||||||
|
assertExpr("strpos(x, 'o', 1)", 1L);
|
||||||
|
assertExpr("strpos(x, 'o', 2)", 2L);
|
||||||
|
assertExpr("strpos(x, 'o', 3)", -1L);
|
||||||
assertExpr("strpos(x, '')", 0L);
|
assertExpr("strpos(x, '')", 0L);
|
||||||
assertExpr("strpos(x, 'x')", -1L);
|
assertExpr("strpos(x, 'x')", -1L);
|
||||||
}
|
}
|
||||||
|
|
|
@ -71,7 +71,7 @@ The following built-in functions are available.
|
||||||
|replace|replace(expr, pattern, replacement) replaces pattern with replacement|
|
|replace|replace(expr, pattern, replacement) replaces pattern with replacement|
|
||||||
|substring|substring(expr, index, length) behaves like java.lang.String's substring|
|
|substring|substring(expr, index, length) behaves like java.lang.String's substring|
|
||||||
|strlen|strlen(expr) returns length of a string in UTF-16 code units|
|
|strlen|strlen(expr) returns length of a string in UTF-16 code units|
|
||||||
|strpos|strpos(haystack, needle) returns the position of the needle within the haystack, with indexes starting from 0. If the needle is not found then the function returns -1.|
|
|strpos|strpos(haystack, needle[, fromIndex]) returns the position of the needle within the haystack, with indexes starting from 0. The search will begin at fromIndex, or 0 if fromIndex is not specified. If the needle is not found then the function returns -1.|
|
||||||
|trim|trim(expr[, chars]) remove leading and trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
|trim|trim(expr[, chars]) remove leading and trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
||||||
|ltrim|ltrim(expr[, chars]) remove leading characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
|ltrim|ltrim(expr[, chars]) remove leading characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
||||||
|rtrim|rtrim(expr[, chars]) remove trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
|rtrim|rtrim(expr[, chars]) remove trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|
||||||
|
|
|
@ -156,9 +156,10 @@ String functions accept strings, and return a type appropriate to the function.
|
||||||
|`STRLEN(expr)`|Synonym for `LENGTH`.|
|
|`STRLEN(expr)`|Synonym for `LENGTH`.|
|
||||||
|`LOOKUP(expr, lookupName)`|Look up expr in a registered [query-time lookup table](lookups.html).|
|
|`LOOKUP(expr, lookupName)`|Look up expr in a registered [query-time lookup table](lookups.html).|
|
||||||
|`LOWER(expr)`|Returns expr in all lowercase.|
|
|`LOWER(expr)`|Returns expr in all lowercase.|
|
||||||
|
|`POSITION(needle IN haystack [FROM fromIndex])`|Returns the index of needle within haystack, with indexes starting from 1. The search will begin at fromIndex, or 1 if fromIndex is not specified. If the needle is not found, returns 0.|
|
||||||
|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression pattern and extract a capture group, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern.|
|
|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression pattern and extract a capture group, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern.|
|
||||||
|`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in expr, and returns the result.|
|
|`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in expr, and returns the result.|
|
||||||
|`STRPOS(haystack, needle)`|Returns the index of needle within haystack, starting from 1. If the needle is not found, returns 0.|
|
|`STRPOS(haystack, needle)`|Returns the index of needle within haystack, with indexes starting from 1. If the needle is not found, returns 0.|
|
||||||
|`SUBSTRING(expr, index, [length])`|Returns a substring of expr starting at index, with a max length, both measured in UTF-16 code units.|
|
|`SUBSTRING(expr, index, [length])`|Returns a substring of expr starting at index, with a max length, both measured in UTF-16 code units.|
|
||||||
|`SUBSTR(expr, index, [length])`|Synonym for SUBSTRING.|
|
|`SUBSTR(expr, index, [length])`|Synonym for SUBSTRING.|
|
||||||
|`TRIM([BOTH \| LEADING \| TRAILING] [<chars> FROM] expr)`|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".|
|
|`TRIM([BOTH \| LEADING \| TRAILING] [<chars> FROM] expr)`|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".|
|
||||||
|
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* 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 com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
import org.apache.calcite.sql.SqlOperator;
|
||||||
|
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||||
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||||
|
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.table.RowSignature;
|
||||||
|
|
||||||
|
public class PositionOperatorConversion implements SqlOperatorConversion
|
||||||
|
{
|
||||||
|
private static final DruidExpression ZERO = DruidExpression.fromExpression("0");
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SqlOperator calciteOperator()
|
||||||
|
{
|
||||||
|
return SqlStdOperatorTable.POSITION;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DruidExpression toDruidExpression(
|
||||||
|
final PlannerContext plannerContext,
|
||||||
|
final RowSignature rowSignature,
|
||||||
|
final RexNode rexNode
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return OperatorConversions.convertCall(
|
||||||
|
plannerContext,
|
||||||
|
rowSignature,
|
||||||
|
rexNode,
|
||||||
|
druidExpressions -> {
|
||||||
|
final DruidExpression fromIndexExpression;
|
||||||
|
if (druidExpressions.size() > 2) {
|
||||||
|
fromIndexExpression = DruidExpression.fromExpression(
|
||||||
|
StringUtils.format("(%s - 1)", druidExpressions.get(2).getExpression())
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
fromIndexExpression = ZERO;
|
||||||
|
}
|
||||||
|
|
||||||
|
return DruidExpression.fromExpression(
|
||||||
|
StringUtils.format(
|
||||||
|
"(%s + 1)",
|
||||||
|
DruidExpression.functionCall(
|
||||||
|
"strpos",
|
||||||
|
ImmutableList.of(druidExpressions.get(1), druidExpressions.get(0), fromIndexExpression)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.expression.builtin.ExtractOperatorConversion
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.FloorOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.FloorOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.LTrimOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.LTrimOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.MillisToTimestampOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.MillisToTimestampOperatorConversion;
|
||||||
|
import org.apache.druid.sql.calcite.expression.builtin.PositionOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.RTrimOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.RTrimOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion;
|
import org.apache.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion;
|
||||||
|
@ -147,6 +148,7 @@ public class DruidOperatorTable implements SqlOperatorTable
|
||||||
.add(new MillisToTimestampOperatorConversion())
|
.add(new MillisToTimestampOperatorConversion())
|
||||||
.add(new ReinterpretOperatorConversion())
|
.add(new ReinterpretOperatorConversion())
|
||||||
.add(new RegexpExtractOperatorConversion())
|
.add(new RegexpExtractOperatorConversion())
|
||||||
|
.add(new PositionOperatorConversion())
|
||||||
.add(new StrposOperatorConversion())
|
.add(new StrposOperatorConversion())
|
||||||
.add(new SubstringOperatorConversion())
|
.add(new SubstringOperatorConversion())
|
||||||
.add(new ConcatOperatorConversion())
|
.add(new ConcatOperatorConversion())
|
||||||
|
|
|
@ -198,6 +198,42 @@ public class ExpressionsTest extends CalciteTestBase
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPosition()
|
||||||
|
{
|
||||||
|
testExpression(
|
||||||
|
rexBuilder.makeCall(
|
||||||
|
SqlStdOperatorTable.POSITION,
|
||||||
|
rexBuilder.makeLiteral("oo"),
|
||||||
|
inputRef("s")
|
||||||
|
),
|
||||||
|
DruidExpression.fromExpression("(strpos(\"s\",'oo',0) + 1)"),
|
||||||
|
2L
|
||||||
|
);
|
||||||
|
|
||||||
|
testExpression(
|
||||||
|
rexBuilder.makeCall(
|
||||||
|
SqlStdOperatorTable.POSITION,
|
||||||
|
rexBuilder.makeLiteral("oo"),
|
||||||
|
inputRef("s"),
|
||||||
|
rexBuilder.makeExactLiteral(BigDecimal.valueOf(2))
|
||||||
|
),
|
||||||
|
DruidExpression.fromExpression("(strpos(\"s\",'oo',(2 - 1)) + 1)"),
|
||||||
|
2L
|
||||||
|
);
|
||||||
|
|
||||||
|
testExpression(
|
||||||
|
rexBuilder.makeCall(
|
||||||
|
SqlStdOperatorTable.POSITION,
|
||||||
|
rexBuilder.makeLiteral("oo"),
|
||||||
|
inputRef("s"),
|
||||||
|
rexBuilder.makeExactLiteral(BigDecimal.valueOf(3))
|
||||||
|
),
|
||||||
|
DruidExpression.fromExpression("(strpos(\"s\",'oo',(3 - 1)) + 1)"),
|
||||||
|
0L
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPower()
|
public void testPower()
|
||||||
{
|
{
|
||||||
|
|
Loading…
Reference in New Issue