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:
Gian Merlino 2018-11-13 13:39:00 -08:00 committed by Jonathan Wei
parent f2b73f9df1
commit 154b6fbcef
7 changed files with 133 additions and 5 deletions

View File

@ -955,8 +955,8 @@ interface Function
@Override
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
{
if (args.size() != 2) {
throw new IAE("Function[%s] needs 2 arguments", name());
if (args.size() < 2 || args.size() > 3) {
throw new IAE("Function[%s] needs 2 or 3 arguments", name());
}
final String haystack = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString());
@ -965,7 +965,16 @@ interface Function
if (haystack == null || needle == 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));
}
}

View File

@ -95,6 +95,10 @@ public class FunctionTest
public void testStrpos()
{
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, 'x')", -1L);
}

View File

@ -71,7 +71,7 @@ The following built-in functions are available.
|replace|replace(expr, pattern, replacement) replaces pattern with replacement|
|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|
|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.|
|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.|

View File

@ -156,9 +156,10 @@ String functions accept strings, and return a type appropriate to the function.
|`STRLEN(expr)`|Synonym for `LENGTH`.|
|`LOOKUP(expr, lookupName)`|Look up expr in a registered [query-time lookup table](lookups.html).|
|`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.|
|`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.|
|`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".|

View File

@ -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)
)
)
);
}
);
}
}

View File

@ -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.LTrimOperatorConversion;
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.RegexpExtractOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion;
@ -147,6 +148,7 @@ public class DruidOperatorTable implements SqlOperatorTable
.add(new MillisToTimestampOperatorConversion())
.add(new ReinterpretOperatorConversion())
.add(new RegexpExtractOperatorConversion())
.add(new PositionOperatorConversion())
.add(new StrposOperatorConversion())
.add(new SubstringOperatorConversion())
.add(new ConcatOperatorConversion())

View File

@ -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
public void testPower()
{