Put sleep in an extension (#11632)

* Put sleep in an extension

* dependency
This commit is contained in:
Jihoon Son 2021-08-25 01:27:45 -07:00 committed by GitHub
parent 1d5438ae7c
commit 2a658acad4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 550 additions and 122 deletions

View File

@ -142,7 +142,7 @@ public interface Expr extends Cacheable
* the entire expression. Otherwise, all vectorizable expressions must produce an output type to correctly operate * the entire expression. Otherwise, all vectorizable expressions must produce an output type to correctly operate
* with the vectorized engine. * with the vectorized engine.
* *
* Outside of the context of vectorized expressions, a return value of null can also indicate that the given type * Outside the context of vectorized expressions, a return value of null can also indicate that the given type
* information was not enough to resolve the output type, so the expression must be evaluated using default * information was not enough to resolve the output type, so the expression must be evaluated using default
* {@link #eval} handling where types are only known after evaluation, through {@link ExprEval#type}, such as * {@link #eval} handling where types are only known after evaluation, through {@link ExprEval#type}, such as
* transform expressions at ingestion time * transform expressions at ingestion time

View File

@ -19,7 +19,6 @@
package org.apache.druid.math.expr; package org.apache.druid.math.expr;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
@ -28,8 +27,6 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.Expr.InputBindingInspector;
import org.apache.druid.math.expr.Expr.ObjectBinding;
import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor; import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorMathProcessors; import org.apache.druid.math.expr.vector.VectorMathProcessors;
@ -3642,56 +3639,4 @@ public interface Function
return HumanReadableBytes.UnitSystem.DECIMAL; return HumanReadableBytes.UnitSystem.DECIMAL;
} }
} }
/**
* This function makes the current thread sleep for the given amount of seconds.
* Fractional-second delays can be specified.
*
* This function is applied per row. The actual query time can vary depending on how much parallelism is used
* for the query. As it does not provide consistent sleep time, this function should be used only for testing
* when you want to keep a certain query running during the test.
*/
@VisibleForTesting
class Sleep implements Function
{
@Override
public String name()
{
return "sleep";
}
@Override
public ExprEval apply(List<Expr> args, ObjectBinding bindings)
{
ExprEval eval = args.get(0).eval(bindings);
try {
if (!eval.isNumericNull()) {
double seconds = eval.asDouble();
if (seconds > 0) {
Thread.sleep((long) (seconds * 1000));
}
}
return ExprEval.of(null);
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
@Override
public void validateArguments(List<Expr> args)
{
if (args.size() != 1) {
throw new IAE("Function[%s] needs 1 argument", name());
}
}
@Nullable
@Override
public ExprType getOutputType(InputBindingInspector inspector, List<Expr> args)
{
return ExprType.STRING;
}
}
} }

View File

@ -783,37 +783,6 @@ public class FunctionTest extends InitializedNullHandlingTest
assertExpr("repeat(nonexistent, 10)", null); assertExpr("repeat(nonexistent, 10)", null);
} }
@Test
public void testSleep()
{
assertExpr("sleep(1)", null);
assertExpr("sleep(0.5)", null);
assertExpr("sleep(null)", null);
assertExpr("sleep(0)", null);
assertExpr("sleep(-1)", null);
assertTimeElapsed("sleep(1)", 1000);
assertTimeElapsed("sleep(0.5)", 500);
assertTimeElapsed("sleep(null)", 0);
assertTimeElapsed("sleep(0)", 0);
assertTimeElapsed("sleep(-1)", 0);
}
private void assertTimeElapsed(String expression, long expectedTimeElapsedMs)
{
final long detla = 50;
final long before = System.currentTimeMillis();
final Expr expr = Parser.parse(expression, ExprMacroTable.nil());
expr.eval(bindings).value();
final long after = System.currentTimeMillis();
final long elapsed = after - before;
Assert.assertTrue(
StringUtils.format("Expected [%s], but actual elapsed was [%s]", expectedTimeElapsedMs, elapsed),
elapsed >= expectedTimeElapsedMs
&& elapsed < expectedTimeElapsedMs + detla
);
}
private void assertExpr(final String expression, @Nullable final Object expectedResult) private void assertExpr(final String expression, @Nullable final Object expectedResult)
{ {
final Expr expr = Parser.parse(expression, ExprMacroTable.nil()); final Expr expr = Parser.parse(expression, ExprMacroTable.nil());

View File

@ -0,0 +1,178 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.druid.extensions</groupId>
<artifactId>druid-testing-tools</artifactId>
<name>druid-testing-tools</name>
<description>Tools useful for testing</description>
<parent>
<groupId>org.apache.druid</groupId>
<artifactId>druid</artifactId>
<version>0.22.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-core</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-guava</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-joda</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-smile</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.jaxrs</groupId>
<artifactId>jackson-jaxrs-json-provider</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.jaxrs</groupId>
<artifactId>jackson-jaxrs-smile-provider</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
<scope>provided</scope>
</dependency>
<!-- Test Dependencies -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-core</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,46 @@
/*
* 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.guice;
import com.fasterxml.jackson.databind.Module;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.query.expressions.SleepExprMacro;
import org.apache.druid.query.sql.SleepOperatorConversion;
import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
public class SleepModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return Collections.emptyList();
}
@Override
public void configure(Binder binder)
{
SqlBindings.addOperatorConversion(binder, SleepOperatorConversion.class);
ExpressionModule.addExprMacro(binder, SleepExprMacro.class);
}
}

View File

@ -0,0 +1,111 @@
/*
* 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.query.expressions;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr;
import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.query.expression.ExprUtils;
import java.util.List;
/**
* This function makes the current thread sleep for the given amount of seconds.
* Fractional-second delays can be specified.
*
* This function is applied per row. The actual query time can vary depending on how much parallelism is used
* for the query. As it does not provide consistent sleep time, this function should be used only for testing
* when you want to keep a certain query running during the test.
*/
public class SleepExprMacro implements ExprMacro
{
private static final String NAME = "sleep";
@Override
public String name()
{
return NAME;
}
@Override
public Expr apply(List<Expr> args)
{
if (args.size() != 1) {
throw new IAE(ExprUtils.createErrMsg(name(), "must have 1 argument"));
}
Expr arg = args.get(0);
class SleepExpr extends BaseScalarUnivariateMacroFunctionExpr
{
public SleepExpr(Expr arg)
{
super(NAME, arg);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
ExprEval eval = arg.eval(bindings);
try {
if (!eval.isNumericNull()) {
double seconds = eval.asDouble(); // double to support fractional-second.
if (seconds > 0) {
Thread.sleep((long) (seconds * 1000));
}
}
return ExprEval.of(null);
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
@Override
public Expr visit(Shuttle shuttle)
{
Expr newArg = arg.visit(shuttle);
return shuttle.visit(new SleepExpr(newArg));
}
/**
* Explicitly override this method to not vectorize the sleep expression.
* If we ever want to vectorize this expression, {@link #getOutputType} should be considered to return something
* else than just null.
*/
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return false;
}
@Override
public ExprType getOutputType(InputBindingInspector inspector)
{
return null;
}
}
return new SleepExpr(arg);
}
}

View File

@ -17,7 +17,7 @@
* under the License. * under the License.
*/ */
package org.apache.druid.sql.calcite.expression.builtin; package org.apache.druid.query.sql;
import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlFunction;
@ -34,7 +34,7 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable; import javax.annotation.Nullable;
/** /**
* A SQL operator conversion for the {@link org.apache.druid.math.expr.Function.Sleep} expression. * A SQL operator conversion for the {@link org.apache.druid.query.expressions.SleepExprMacro} expression.
* The expression is currently evaluated during the query planning when the given argument is a number literal. * The expression is currently evaluated during the query planning when the given argument is a number literal.
*/ */
public class SleepOperatorConversion implements SqlOperatorConversion public class SleepOperatorConversion implements SqlOperatorConversion

View File

@ -0,0 +1,16 @@
# 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.
org.apache.druid.guice.SleepModule

View File

@ -0,0 +1,96 @@
/*
* 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.query.expressions;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.Expr.ObjectBinding;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Collections;
public class SleepExprTest extends InitializedNullHandlingTest
{
private final ObjectBinding bindings = new ObjectBinding()
{
@Nullable
@Override
public Object get(String name)
{
return null;
}
};
private final ExprMacroTable exprMacroTable = new ExprMacroTable(Collections.singletonList(new SleepExprMacro()));
@Test
public void testSleep()
{
assertExpr("sleep(1)");
assertExpr("sleep(0.5)");
assertExpr("sleep(null)");
assertExpr("sleep(0)");
assertExpr("sleep(-1)");
assertTimeElapsed("sleep(1)", 1000);
assertTimeElapsed("sleep(0.5)", 500);
assertTimeElapsed("sleep(null)", 0);
assertTimeElapsed("sleep(0)", 0);
assertTimeElapsed("sleep(-1)", 0);
}
private void assertTimeElapsed(String expression, long expectedTimeElapsedMs)
{
final long detla = 50;
final long before = System.currentTimeMillis();
final Expr expr = Parser.parse(expression, exprMacroTable);
expr.eval(bindings).value();
final long after = System.currentTimeMillis();
final long elapsed = after - before;
Assert.assertTrue(
StringUtils.format("Expected [%s], but actual elapsed was [%s]", expectedTimeElapsedMs, elapsed),
elapsed >= expectedTimeElapsedMs
&& elapsed < expectedTimeElapsedMs + detla
);
}
private void assertExpr(final String expression)
{
final Expr expr = Parser.parse(expression, exprMacroTable);
Assert.assertNull(expression, expr.eval(bindings).value());
final Expr exprNoFlatten = Parser.parse(expression, exprMacroTable, false);
final Expr roundTrip = Parser.parse(exprNoFlatten.stringify(), exprMacroTable);
Assert.assertNull(expr.stringify(), roundTrip.eval(bindings).value());
final Expr roundTripFlatten = Parser.parse(expr.stringify(), exprMacroTable);
Assert.assertNull(expr.stringify(), roundTripFlatten.eval(bindings).value());
Assert.assertEquals(expr.stringify(), roundTrip.stringify());
Assert.assertEquals(expr.stringify(), roundTripFlatten.stringify());
Assert.assertArrayEquals(expr.getCacheKey(), roundTrip.getCacheKey());
Assert.assertArrayEquals(expr.getCacheKey(), roundTripFlatten.getCacheKey());
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.query.sql;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.ExpressionModule;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
import org.apache.druid.query.Druids;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.expression.LookupExprMacro;
import org.apache.druid.query.expressions.SleepExprMacro;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.scan.ScanQuery.ResultFormat;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class SleepSqlTest extends BaseCalciteQueryTest
{
@Override
public DruidOperatorTable createOperatorTable()
{
return new DruidOperatorTable(
ImmutableSet.of(),
ImmutableSet.of(new SleepOperatorConversion())
);
}
@Override
public ExprMacroTable createMacroTable()
{
final List<ExprMacro> exprMacros = new ArrayList<>();
for (Class<? extends ExprMacroTable.ExprMacro> clazz : ExpressionModule.EXPR_MACROS) {
exprMacros.add(CalciteTests.INJECTOR.getInstance(clazz));
}
exprMacros.add(CalciteTests.INJECTOR.getInstance(LookupExprMacro.class));
exprMacros.add(new SleepExprMacro());
return new ExprMacroTable(exprMacros);
}
@Test
public void testSleepFunction() throws Exception
{
testQuery(
"SELECT sleep(m1) from foo where m1 < 2.0",
ImmutableList.of(
Druids.newScanQueryBuilder()
.dataSource(new TableDataSource("foo"))
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
new ExpressionVirtualColumn(
"v0",
"sleep(\"m1\")",
ValueType.STRING,
createMacroTable()
)
)
.columns("v0")
.filters(new BoundDimFilter("m1", null, "2.0", null, true, null, null, StringComparators.NUMERIC))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.replaceWithDefault() ? "" : null}
)
);
}
}

View File

@ -170,6 +170,7 @@
<module>extensions-core/druid-basic-security</module> <module>extensions-core/druid-basic-security</module>
<module>extensions-core/google-extensions</module> <module>extensions-core/google-extensions</module>
<module>extensions-core/druid-ranger-security</module> <module>extensions-core/druid-ranger-security</module>
<module>extensions-core/testing-tools</module>
<!-- Community extensions --> <!-- Community extensions -->
<module>extensions-contrib/influx-extensions</module> <module>extensions-contrib/influx-extensions</module>
<module>extensions-contrib/cassandra-storage</module> <module>extensions-contrib/cassandra-storage</module>

View File

@ -88,7 +88,7 @@ public class ExprUtils
return new PeriodGranularity(period, origin, timeZone); return new PeriodGranularity(period, origin, timeZone);
} }
static String createErrMsg(String functionName, String msg) public static String createErrMsg(String functionName, String msg)
{ {
String prefix = "Function[" + functionName + "] "; String prefix = "Function[" + functionName + "] ";
return prefix + msg; return prefix + msg;

View File

@ -97,7 +97,6 @@ import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.RoundOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RoundOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.SleepOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.StringFormatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.StringFormatOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.StringToArrayOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.StringToArrayOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.StrposOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.StrposOperatorConversion;
@ -169,7 +168,6 @@ public class DruidOperatorTable implements SqlOperatorTable
.add(new TimeParseOperatorConversion()) .add(new TimeParseOperatorConversion())
.add(new TimeShiftOperatorConversion()) .add(new TimeShiftOperatorConversion())
.add(new TimestampToMillisOperatorConversion()) .add(new TimestampToMillisOperatorConversion())
.add(new SleepOperatorConversion())
.build(); .build();
private static final List<SqlOperatorConversion> STRING_OPERATOR_CONVERSIONS = private static final List<SqlOperatorConversion> STRING_OPERATOR_CONVERSIONS =

View File

@ -18858,34 +18858,4 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of() ImmutableList.of()
); );
} }
@Test
public void testSleepFunction() throws Exception
{
testQuery(
"SELECT sleep(m1) from foo where m1 < 2.0",
ImmutableList.of(
Druids.newScanQueryBuilder()
.dataSource(new TableDataSource("foo"))
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
new ExpressionVirtualColumn(
"v0",
"sleep(\"m1\")",
ValueType.STRING,
ExprMacroTable.nil()
)
)
.columns("v0")
.filters(new BoundDimFilter("m1", null, "2.0", null, true, null, null, StringComparators.NUMERIC))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.replaceWithDefault() ? "" : null}
)
);
}
} }