diff --git a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java index 850e5a44ccd..8eb67196cc0 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java @@ -411,4 +411,80 @@ public class StringUtils System.arraycopy(multiple, 0, multiple, copied, limit - copied); return new String(multiple, StandardCharsets.UTF_8); } + + /** + * Returns the string left-padded with the string pad to a length of len characters. + * If str is longer than len, the return value is shortened to len characters. + * Lpad and rpad functions are migrated from flink's scala function with minor refactor + * https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala + * + * @param base The base string to be padded + * @param len The length of padded string + * @param pad The pad string + * @return the string left-padded with pad to a length of len + */ + public static String lpad(String base, Integer len, String pad) + { + if (len < 0) { + return null; + } else if (len == 0) { + return ""; + } + + char[] data = new char[len]; + + // The length of the padding needed + int pos = Math.max(len - base.length(), 0); + + // Copy the padding + for (int i = 0; i < pos; i += pad.length()) { + for (int j = 0; j < pad.length() && j < pos - i; j++) { + data[i + j] = pad.charAt(j); + } + } + + // Copy the base + for (int i = 0; pos + i < len && i < base.length(); i++) { + data[pos + i] = base.charAt(i); + } + + return new String(data); + } + + /** + * Returns the string right-padded with the string pad to a length of len characters. + * If str is longer than len, the return value is shortened to len characters. + * + * @param base The base string to be padded + * @param len The length of padded string + * @param pad The pad string + * @return the string right-padded with pad to a length of len + */ + public static String rpad(String base, Integer len, String pad) + { + if (len < 0) { + return null; + } else if (len == 0) { + return ""; + } + + char[] data = new char[len]; + + int pos = 0; + + // Copy the base + for ( ; pos < base.length() && pos < len; pos++) { + data[pos] = base.charAt(pos); + } + + // Copy the padding + for ( ; pos < len; pos += pad.length()) { + for (int i = 0; i < pad.length() && i < len - pos; i++) { + data[pos + i] = pad.charAt(i); + } + } + + return new String(data); + } + } diff --git a/core/src/main/java/org/apache/druid/math/expr/Function.java b/core/src/main/java/org/apache/druid/math/expr/Function.java index 4c8cd93d8ba..31cdd8e6ca2 100644 --- a/core/src/main/java/org/apache/druid/math/expr/Function.java +++ b/core/src/main/java/org/apache/druid/math/expr/Function.java @@ -1367,4 +1367,61 @@ interface Function return ExprEval.of(expr.value() != null, ExprType.LONG); } } + + class LpadFunc implements Function + { + @Override + public String name() + { + return "lpad"; + } + + @Override + public ExprEval apply(List args, Expr.ObjectBinding bindings) + { + if (args.size() != 3) { + throw new IAE("Function[%s] needs 3 arguments", name()); + } + + String base = args.get(0).eval(bindings).asString(); + int len = args.get(1).eval(bindings).asInt(); + String pad = args.get(2).eval(bindings).asString(); + + if (base == null || pad == null) { + return ExprEval.of(null); + } else { + return ExprEval.of(len == 0 ? NullHandling.defaultStringValue() : StringUtils.lpad(base, len, pad)); + } + + } + } + + class RpadFunc implements Function + { + @Override + public String name() + { + return "rpad"; + } + + @Override + public ExprEval apply(List args, Expr.ObjectBinding bindings) + { + if (args.size() != 3) { + throw new IAE("Function[%s] needs 3 arguments", name()); + } + + String base = args.get(0).eval(bindings).asString(); + int len = args.get(1).eval(bindings).asInt(); + String pad = args.get(2).eval(bindings).asString(); + + if (base == null || pad == null) { + return ExprEval.of(null); + } else { + return ExprEval.of(len == 0 ? NullHandling.defaultStringValue() : StringUtils.rpad(base, len, pad)); + } + + } + } + } diff --git a/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java b/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java index 6f5a3b553c7..8a1748cc249 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java @@ -181,4 +181,43 @@ public class StringUtilsTest expectedException.expectMessage("count is negative, -1"); Assert.assertEquals("", StringUtils.repeat("foo", -1)); } + + @Test + public void testLpad() + { + String s1 = StringUtils.lpad("abc", 7, "de"); + Assert.assertEquals(s1, "dedeabc"); + + String s2 = StringUtils.lpad("abc", 6, "de"); + Assert.assertEquals(s2, "dedabc"); + + String s3 = StringUtils.lpad("abc", 2, "de"); + Assert.assertEquals(s3, "ab"); + + String s4 = StringUtils.lpad("abc", 0, "de"); + Assert.assertEquals(s4, ""); + + String s5 = StringUtils.lpad("abc", -1, "de"); + Assert.assertEquals(s5, null); + } + + @Test + public void testRpad() + { + String s1 = StringUtils.rpad("abc", 7, "de"); + Assert.assertEquals(s1, "abcdede"); + + String s2 = StringUtils.rpad("abc", 6, "de"); + Assert.assertEquals(s2, "abcded"); + + String s3 = StringUtils.rpad("abc", 2, "de"); + Assert.assertEquals(s3, "ab"); + + String s4 = StringUtils.rpad("abc", 0, "de"); + Assert.assertEquals(s4, ""); + + String s5 = StringUtils.rpad("abc", -1, "de"); + Assert.assertEquals(s5, null); + } + } diff --git a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java index bc042834087..2a5bfc95cc0 100644 --- a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -134,4 +134,26 @@ public class FunctionTest assertExpr("notnull(null)", 0L); assertExpr("notnull('abc')", 1L); } + + @Test + public void testLpad() + { + assertExpr("lpad(x, 5, 'ab')", "abfoo"); + assertExpr("lpad(x, 4, 'ab')", "afoo"); + assertExpr("lpad(x, 2, 'ab')", "fo"); + assertExpr("lpad(x, 0, 'ab')", null); + assertExpr("lpad(x, 5, null)", null); + assertExpr("lpad(null, 5, x)", null); + } + + @Test + public void testRpad() + { + assertExpr("rpad(x, 5, 'ab')", "fooab"); + assertExpr("rpad(x, 4, 'ab')", "fooa"); + assertExpr("rpad(x, 2, 'ab')", "fo"); + assertExpr("rpad(x, 0, 'ab')", null); + assertExpr("rpad(x, 5, null)", null); + assertExpr("rpad(null, 5, x)", null); + } } diff --git a/docs/content/misc/math-expr.md b/docs/content/misc/math-expr.md index fc71c70403b..1358cc4b94d 100644 --- a/docs/content/misc/math-expr.md +++ b/docs/content/misc/math-expr.md @@ -85,6 +85,8 @@ The following built-in functions are available. |upper|upper(expr) converts a string to uppercase| |reverse|reverse(expr) reverses a string| |repeat|repeat(expr, N) repeats a string N times| +|lpad|lpad(expr, length, chars) returns a string of `length` from `expr` left-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. If either `expr` or `chars` are null, the result will be null.| +|rpad|rpad(expr, length, chars) returns a string of `length` from `expr` right-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. If either `expr` or `chars` are null, the result will be null.| ## Time functions diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index 519fdd762f4..cb8ea88a1fb 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -198,6 +198,9 @@ String functions accept strings, and return a type appropriate to the function. |`UPPER(expr)`|Returns expr in all uppercase.| |`REVERSE(expr)`|Reverses expr.| |`REPEAT(expr, [N])`|Repeats expr N times| +|`LPAD(expr, length[, chars])`|Returns a string of "length" from "expr" left-padded with "chars". If "length" is shorter than the length of "expr", the result is "expr" which is truncated to "length". If either "expr" or "chars" are null, the result will be null.| +|`RPAD(expr, length[, chars])`|Returns a string of "length" from "expr" right-padded with "chars". If "length" is shorter than the length of "expr", the result is "expr" which is truncated to "length". If either "expr" or "chars" are null, the result will be null.| + ### Time functions diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java new file mode 100644 index 00000000000..f7fdb12ded1 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java @@ -0,0 +1,85 @@ +/* + * 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.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +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 LPadOperatorConversion implements SqlOperatorConversion +{ + private static final SqlFunction SQL_FUNCTION = OperatorConversions + .operatorBuilder("LPAD") + .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) + .returnType(SqlTypeName.VARCHAR) + .functionCategory(SqlFunctionCategory.STRING) + .requiredOperands(2) + .build(); + + @Override + public SqlOperator calciteOperator() + { + return SQL_FUNCTION; + } + + @Override + public DruidExpression toDruidExpression( + final PlannerContext plannerContext, + final RowSignature rowSignature, + final RexNode rexNode + ) + { + return OperatorConversions.convertCall( + plannerContext, + rowSignature, + rexNode, + druidExpressions -> { + if (druidExpressions.size() > 2) { + return DruidExpression.fromFunctionCall( + "lpad", + ImmutableList.of( + druidExpressions.get(0), + druidExpressions.get(1), + druidExpressions.get(2) + ) + ); + } else { + return DruidExpression.fromFunctionCall( + "lpad", + ImmutableList.of( + druidExpressions.get(0), + druidExpressions.get(1), + DruidExpression.fromExpression(DruidExpression.stringLiteral(" ")) + ) + ); + } + } + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java new file mode 100644 index 00000000000..e2bc5d2b81f --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java @@ -0,0 +1,85 @@ +/* + * 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.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +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 RPadOperatorConversion implements SqlOperatorConversion +{ + private static final SqlFunction SQL_FUNCTION = OperatorConversions + .operatorBuilder("RPAD") + .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) + .returnType(SqlTypeName.VARCHAR) + .functionCategory(SqlFunctionCategory.STRING) + .requiredOperands(2) + .build(); + + @Override + public SqlOperator calciteOperator() + { + return SQL_FUNCTION; + } + + @Override + public DruidExpression toDruidExpression( + final PlannerContext plannerContext, + final RowSignature rowSignature, + final RexNode rexNode + ) + { + return OperatorConversions.convertCall( + plannerContext, + rowSignature, + rexNode, + druidExpressions -> { + if (druidExpressions.size() > 2) { + return DruidExpression.fromFunctionCall( + "rpad", + ImmutableList.of( + druidExpressions.get(0), + druidExpressions.get(1), + druidExpressions.get(2) + ) + ); + } else { + return DruidExpression.fromFunctionCall( + "rpad", + ImmutableList.of( + druidExpressions.get(0), + druidExpressions.get(1), + DruidExpression.fromExpression(DruidExpression.stringLiteral(" ")) + ) + ); + } + } + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index 82682f56549..7ebd03b1311 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -53,12 +53,14 @@ import org.apache.druid.sql.calcite.expression.builtin.ConcatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion; 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.LPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LTrimOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LeftOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LikeOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.MillisToTimestampOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ParseLongOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.PositionOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; 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; @@ -196,6 +198,8 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new TrimOperatorConversion()) .add(new TruncateOperatorConversion()) .add(new AliasedOperatorConversion(new TruncateOperatorConversion(), "TRUNC")) + .add(new LPadOperatorConversion()) + .add(new RPadOperatorConversion()) // value coercion operators .add(new CastOperatorConversion()) .add(new ReinterpretOperatorConversion()) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 1e4d799a299..6eb18824cff 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -236,6 +236,43 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testSelectPadFamily() throws Exception + { + testQuery( + "SELECT\n" + + "LPAD('foo', 5, 'x'),\n" + + "LPAD('foo', 2, 'x'),\n" + + "LPAD('foo', 5),\n" + + "RPAD('foo', 5, 'x'),\n" + + "RPAD('foo', 2, 'x'),\n" + + "RPAD('foo', 5),\n" + + "COUNT(*)\n" + + "FROM foo", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .postAggregators( + expressionPostAgg("p0", "'xxfoo'"), + expressionPostAgg("p1", "'fo'"), + expressionPostAgg("p2", "' foo'"), + expressionPostAgg("p3", "'fooxx'"), + expressionPostAgg("p4", "'fo'"), + expressionPostAgg("p5", "'foo '") + ) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"xxfoo", "fo", " foo", "fooxx", "fo", "foo ", 6L} + ) + ); + } + + @Test public void testExplainSelectConstantExpression() throws Exception { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index ee6f616dd26..5bf49235a6c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -42,8 +42,10 @@ import org.apache.druid.math.expr.Parser; import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.LPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LeftOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ParseLongOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; @@ -601,6 +603,33 @@ public class ExpressionsTest extends CalciteTestBase ); } + @Test + public void testPad() + { + testExpression( + rexBuilder.makeCall( + new LPadOperatorConversion().calciteOperator(), + inputRef("s"), + rexBuilder.makeLiteral(5, typeFactory.createSqlType(SqlTypeName.INTEGER), true), + rexBuilder.makeLiteral("x") + ), + DruidExpression.fromExpression("lpad(\"s\",5,'x')"), + "xxfoo" + ); + + testExpression( + rexBuilder.makeCall( + new RPadOperatorConversion().calciteOperator(), + inputRef("s"), + rexBuilder.makeLiteral(5, typeFactory.createSqlType(SqlTypeName.INTEGER), true), + rexBuilder.makeLiteral("x") + ), + DruidExpression.fromExpression("rpad(\"s\",5,'x')"), + "fooxx" + ); + } + + @Test public void testTimeFloor() {