You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2021/08/25 08:28:10 UTC

[druid] branch master updated: Put sleep in an extension (#11632)

This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a658ac  Put sleep in an extension (#11632)
2a658ac is described below

commit 2a658acad4cd419951d2d9d3f17a67cc26f18727
Author: Jihoon Son <ji...@apache.org>
AuthorDate: Wed Aug 25 01:27:45 2021 -0700

    Put sleep in an extension (#11632)
    
    * Put sleep in an extension
    
    * dependency
---
 .../main/java/org/apache/druid/math/expr/Expr.java |   2 +-
 .../java/org/apache/druid/math/expr/Function.java  |  55 -------
 .../org/apache/druid/math/expr/FunctionTest.java   |  31 ----
 extensions-core/testing-tools/pom.xml              | 178 +++++++++++++++++++++
 .../java/org/apache/druid/guice/SleepModule.java   |  46 ++++++
 .../druid/query/expressions/SleepExprMacro.java    | 111 +++++++++++++
 .../druid/query/sql}/SleepOperatorConversion.java  |   4 +-
 .../org.apache.druid.initialization.DruidModule    |  16 ++
 .../druid/query/expressions/SleepExprTest.java     |  96 +++++++++++
 .../org/apache/druid/query/sql/SleepSqlTest.java   |  98 ++++++++++++
 pom.xml                                            |   1 +
 .../apache/druid/query/expression/ExprUtils.java   |   2 +-
 .../sql/calcite/planner/DruidOperatorTable.java    |   2 -
 .../apache/druid/sql/calcite/CalciteQueryTest.java |  30 ----
 14 files changed, 550 insertions(+), 122 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/math/expr/Expr.java b/core/src/main/java/org/apache/druid/math/expr/Expr.java
index a61a6d5..f277179 100644
--- a/core/src/main/java/org/apache/druid/math/expr/Expr.java
+++ b/core/src/main/java/org/apache/druid/math/expr/Expr.java
@@ -142,7 +142,7 @@ public interface Expr extends Cacheable
    * the entire expression. Otherwise, all vectorizable expressions must produce an output type to correctly operate
    * 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
    * {@link #eval} handling where types are only known after evaluation, through {@link ExprEval#type}, such as
    * transform expressions at ingestion time
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 cd677b2..9e4a241 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
@@ -19,7 +19,6 @@
 
 package org.apache.druid.math.expr;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 import org.apache.druid.common.config.NullHandling;
 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.StringUtils;
 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.ExprVectorProcessor;
 import org.apache.druid.math.expr.vector.VectorMathProcessors;
@@ -3642,56 +3639,4 @@ public interface Function
       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;
-    }
-  }
 }
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 99d4753..a5a9fce 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
@@ -783,37 +783,6 @@ public class FunctionTest extends InitializedNullHandlingTest
     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)
   {
     final Expr expr = Parser.parse(expression, ExprMacroTable.nil());
diff --git a/extensions-core/testing-tools/pom.xml b/extensions-core/testing-tools/pom.xml
new file mode 100644
index 0000000..c290baa
--- /dev/null
+++ b/extensions-core/testing-tools/pom.xml
@@ -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>
diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/SleepModule.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/SleepModule.java
new file mode 100644
index 0000000..ec82319
--- /dev/null
+++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/SleepModule.java
@@ -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);
+  }
+}
diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
new file mode 100644
index 0000000..924f495
--- /dev/null
+++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
@@ -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);
+  }
+}
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SleepOperatorConversion.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java
similarity index 93%
rename from sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SleepOperatorConversion.java
rename to extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java
index de9d186..c50c051 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SleepOperatorConversion.java
+++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java
@@ -17,7 +17,7 @@
  * 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.sql.SqlFunction;
@@ -34,7 +34,7 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
 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.
  */
 public class SleepOperatorConversion implements SqlOperatorConversion
diff --git a/extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100644
index 0000000..f2ddf3c
--- /dev/null
+++ b/extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -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
diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/expressions/SleepExprTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/expressions/SleepExprTest.java
new file mode 100644
index 0000000..8aa2be7
--- /dev/null
+++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/expressions/SleepExprTest.java
@@ -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());
+  }
+}
diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java
new file mode 100644
index 0000000..8af1e93
--- /dev/null
+++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java
@@ -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}
+        )
+    );
+  }
+}
diff --git a/pom.xml b/pom.xml
index 8989b1a..908abaf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -170,6 +170,7 @@
         <module>extensions-core/druid-basic-security</module>
         <module>extensions-core/google-extensions</module>
         <module>extensions-core/druid-ranger-security</module>
+        <module>extensions-core/testing-tools</module>
         <!-- Community extensions -->
         <module>extensions-contrib/influx-extensions</module>
         <module>extensions-contrib/cassandra-storage</module>
diff --git a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
index 23ef684..184ff53 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
@@ -88,7 +88,7 @@ public class ExprUtils
     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 + "] ";
     return prefix + msg;
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 40f7a58..0f52c34 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
@@ -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.RightOperatorConversion;
 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.StringToArrayOperatorConversion;
 import org.apache.druid.sql.calcite.expression.builtin.StrposOperatorConversion;
@@ -169,7 +168,6 @@ public class DruidOperatorTable implements SqlOperatorTable
                    .add(new TimeParseOperatorConversion())
                    .add(new TimeShiftOperatorConversion())
                    .add(new TimestampToMillisOperatorConversion())
-                   .add(new SleepOperatorConversion())
                    .build();
 
   private static final List<SqlOperatorConversion> STRING_OPERATOR_CONVERSIONS =
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 102e2b2..3afbbb7 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
@@ -18858,34 +18858,4 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
         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}
-        )
-    );
-  }
 }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org