From 7a25ee4fd9a4f1418f5254e5d34bf2261651e214 Mon Sep 17 00:00:00 2001 From: YongGang Date: Wed, 25 Oct 2023 09:55:56 -0700 Subject: [PATCH 01/15] Ability to send task types to k8s or worker task runner (#15196) * Ability to send task types to k8s or worker task runner * add more tests * use runnerStrategy to determine task runner * minor refine * refine runner strategy config * move workerType config to upper level * validate config when application start --- .../extensions-contrib/k8s-jobs.md | 7 +- .../KubernetesAndWorkerTaskRunner.java | 10 +- .../KubernetesAndWorkerTaskRunnerConfig.java | 51 +++---- .../KubernetesAndWorkerTaskRunnerFactory.java | 19 ++- .../overlord/KubernetesOverlordModule.java | 55 +++++++- .../KubernetesRunnerStrategy.java | 43 ++++++ .../runnerstrategy/RunnerStrategy.java | 75 ++++++++++ .../TaskTypeRunnerStrategy.java | 128 ++++++++++++++++++ .../runnerstrategy/WorkerRunnerStrategy.java | 43 ++++++ ...bernetesAndWorkerTaskRunnerConfigTest.java | 10 +- ...ernetesAndWorkerTaskRunnerFactoryTest.java | 11 +- .../KubernetesAndWorkerTaskRunnerTest.java | 36 ++++- .../KubernetesRunnerStrategyTest.java | 43 ++++++ .../TaskTypeRunnerStrategyTest.java | 64 +++++++++ .../WorkerRunnerStrategyTest.java | 43 ++++++ .../kubernetesAndWorkerTaskRunnerConfig.json | 4 +- 16 files changed, 587 insertions(+), 55 deletions(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/RunnerStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategyTest.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategyTest.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategyTest.java diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 2132b55ea1e3..27290a9bee59 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -284,6 +284,9 @@ To do this, set the following property. |Property| Possible Values |Description|Default|required| |--------|-----------------|-----------|-------|--------| -|`druid.indexer.runner.k8sAndWorker.workerTaskRunnerType`|`String`|Determines whether the `httpRemote` or the `remote` task runner should be used in addition to the Kubernetes task runner.|`httpRemote`|No| -|`druid.indexer.runner.k8sAndWorker.sendAllTasksToWorkerTaskRunner`|`boolean`| Whether to send all the tasks to the worker task runner. If this is set to false all tasks will be sent to Kubernetes|`false`|No| +|`druid.indexer.runner.k8sAndWorker.runnerStrategy.type`| `String` (e.g., `k8s`, `worker`, `taskType`)| Defines the strategy for task runner selection. |`k8s`|No| +|`druid.indexer.runner.k8sAndWorker.runnerStrategy.workerType`| `String` (e.g., `httpRemote`, `remote`)| Specifies the variant of the worker task runner to be utilized.|`httpRemote`|No| +| **For `taskType` runner strategy:**||||| +|`druid.indexer.runner.k8sAndWorker.runnerStrategy.taskType.default`| `String` (e.g., `k8s`, `worker`) | Specifies the default runner to use if no overrides apply. This setting ensures there is always a fallback runner available.|None|No| +|`druid.indexer.runner.k8sAndWorker.runnerStrategy.taskType.overrides`| `JsonObject`(e.g., `{"index_kafka": "worker"}`)| Defines task-specific overrides for runner types. Each entry sets a task type to a specific runner, allowing fine control. |`{}`|No| diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunner.java index 8c41772aea69..d8f4e9d84f21 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunner.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.k8s.overlord.runnerstrategy.RunnerStrategy; import org.apache.druid.tasklogs.TaskLogStreamer; import javax.annotation.Nullable; @@ -57,17 +58,17 @@ public class KubernetesAndWorkerTaskRunner implements TaskLogStreamer, WorkerTas { private final KubernetesTaskRunner kubernetesTaskRunner; private final WorkerTaskRunner workerTaskRunner; - private final KubernetesAndWorkerTaskRunnerConfig kubernetesAndWorkerTaskRunnerConfig; + private final RunnerStrategy runnerStrategy; public KubernetesAndWorkerTaskRunner( KubernetesTaskRunner kubernetesTaskRunner, WorkerTaskRunner workerTaskRunner, - KubernetesAndWorkerTaskRunnerConfig kubernetesAndWorkerTaskRunnerConfig + RunnerStrategy runnerStrategy ) { this.kubernetesTaskRunner = kubernetesTaskRunner; this.workerTaskRunner = workerTaskRunner; - this.kubernetesAndWorkerTaskRunnerConfig = kubernetesAndWorkerTaskRunnerConfig; + this.runnerStrategy = runnerStrategy; } @Override @@ -101,7 +102,8 @@ public void unregisterListener(String listenerId) @Override public ListenableFuture run(Task task) { - if (kubernetesAndWorkerTaskRunnerConfig.isSendAllTasksToWorkerTaskRunner()) { + RunnerStrategy.RunnerType runnerType = runnerStrategy.getRunnerTypeForTask(task); + if (RunnerStrategy.RunnerType.WORKER_RUNNER_TYPE.equals(runnerType)) { return workerTaskRunner.run(task); } else { return kubernetesTaskRunner.run(task); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfig.java index 8e6fb8f7c617..0ffeb0103afa 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfig.java @@ -26,56 +26,49 @@ import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; -import javax.validation.constraints.NotNull; +import javax.annotation.Nullable; public class KubernetesAndWorkerTaskRunnerConfig { - private static final String DEFAULT_WORKER_TASK_RUNNER_TYPE = HttpRemoteTaskRunnerFactory.TYPE_NAME; - /** - * Select which worker task runner to use in addition to the Kubernetes runner, options are httpRemote or remote. - */ - @JsonProperty - @NotNull - private final String workerTaskRunnerType; + private final String RUNNER_STRATEGY_TYPE = "runnerStrategy.type"; + private final String RUNNER_STRATEGY_WORKER_TYPE = "runnerStrategy.workerType"; /** - * Whether or not to send tasks to the worker task runner instead of the Kubernetes runner. + * Select which runner type a task would run on, options are k8s or worker. */ - @JsonProperty - @NotNull - private final boolean sendAllTasksToWorkerTaskRunner; + @JsonProperty(RUNNER_STRATEGY_TYPE) + private String runnerStrategy; + + @JsonProperty(RUNNER_STRATEGY_WORKER_TYPE) + private String workerType; @JsonCreator public KubernetesAndWorkerTaskRunnerConfig( - @JsonProperty("workerTaskRunnerType") String workerTaskRunnerType, - @JsonProperty("sendAllTasksToWorkerTaskRunner") Boolean sendAllTasksToWorkerTaskRunner + @JsonProperty(RUNNER_STRATEGY_TYPE) @Nullable String runnerStrategy, + @JsonProperty(RUNNER_STRATEGY_WORKER_TYPE) @Nullable String workerType ) { - this.workerTaskRunnerType = ObjectUtils.defaultIfNull( - workerTaskRunnerType, - DEFAULT_WORKER_TASK_RUNNER_TYPE - ); + this.runnerStrategy = ObjectUtils.defaultIfNull(runnerStrategy, KubernetesTaskRunnerFactory.TYPE_NAME); + this.workerType = ObjectUtils.defaultIfNull(workerType, HttpRemoteTaskRunnerFactory.TYPE_NAME); Preconditions.checkArgument( - this.workerTaskRunnerType.equals(HttpRemoteTaskRunnerFactory.TYPE_NAME) || - this.workerTaskRunnerType.equals(RemoteTaskRunnerFactory.TYPE_NAME), - "workerTaskRunnerType must be set to one of (%s, %s)", + this.workerType.equals(HttpRemoteTaskRunnerFactory.TYPE_NAME) || + this.workerType.equals(RemoteTaskRunnerFactory.TYPE_NAME), + "workerType must be set to one of (%s, %s)", HttpRemoteTaskRunnerFactory.TYPE_NAME, RemoteTaskRunnerFactory.TYPE_NAME ); - this.sendAllTasksToWorkerTaskRunner = ObjectUtils.defaultIfNull( - sendAllTasksToWorkerTaskRunner, - false - ); } - public String getWorkerTaskRunnerType() + @JsonProperty(RUNNER_STRATEGY_TYPE) + public String getRunnerStrategy() { - return workerTaskRunnerType; + return runnerStrategy; } - public boolean isSendAllTasksToWorkerTaskRunner() + @JsonProperty(RUNNER_STRATEGY_WORKER_TYPE) + public String getWorkerType() { - return sendAllTasksToWorkerTaskRunner; + return workerType; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactory.java index 49ca454f50ae..de6db915c8a9 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactory.java @@ -22,7 +22,9 @@ import com.google.inject.Inject; import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskRunnerFactory; +import org.apache.druid.indexing.overlord.WorkerTaskRunner; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; +import org.apache.druid.k8s.overlord.runnerstrategy.RunnerStrategy; public class KubernetesAndWorkerTaskRunnerFactory implements TaskRunnerFactory @@ -33,6 +35,7 @@ public class KubernetesAndWorkerTaskRunnerFactory implements TaskRunnerFactory + { + private KubernetesAndWorkerTaskRunnerConfig runnerConfig; + private Properties props; + private JsonConfigurator configurator; + + @Inject + public void inject( + KubernetesAndWorkerTaskRunnerConfig runnerConfig, + Properties props, + JsonConfigurator configurator + ) + { + this.runnerConfig = runnerConfig; + this.props = props; + this.configurator = configurator; + } + + @Override + public RunnerStrategy get() + { + String runnerStrategy = runnerConfig.getRunnerStrategy(); + + final String runnerStrategyPropertyBase = StringUtils.format( + RUNNERSTRATEGY_PROPERTIES_FORMAT_STRING, + runnerStrategy + ); + final JsonConfigProvider provider = JsonConfigProvider.of( + runnerStrategyPropertyBase, + RunnerStrategy.class + ); + + props.put(runnerStrategyPropertyBase + ".type", runnerStrategy); + provider.inject(props, configurator); + + return provider.get(); + } + } + private void configureTaskLogs(Binder binder) { PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(FileTaskLogs.class)); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategy.java new file mode 100644 index 000000000000..8b0a6374ad46 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategy.java @@ -0,0 +1,43 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.indexing.common.task.Task; + +/** + * Implementation of {@link RunnerStrategy} that always selects the Kubernetes runner type for tasks. + * + *

This strategy is specific for tasks that are intended to be executed in a Kubernetes environment. + * Regardless of task specifics, this strategy always returns {@link RunnerType#KUBERNETES_RUNNER_TYPE}. + */ +public class KubernetesRunnerStrategy implements RunnerStrategy +{ + @JsonCreator + public KubernetesRunnerStrategy() + { + } + + @Override + public RunnerType getRunnerTypeForTask(Task task) + { + return RunnerType.KUBERNETES_RUNNER_TYPE; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/RunnerStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/RunnerStrategy.java new file mode 100644 index 000000000000..5aa2bc4723ab --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/RunnerStrategy.java @@ -0,0 +1,75 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerFactory; + +/** + * Strategy interface for selecting the appropriate runner type based on the task spec or specific context conditions. + * + *

This interface is part of a strategy pattern and is implemented by different classes that handle + * the logic of selecting a runner type based on various criteria. Each task submitted to the system + * will pass through the strategy implementation to determine the correct runner for execution. + * + *

The strategy uses {@link RunnerType} as a standardized way of referring to and managing different types of runners. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = KubernetesRunnerStrategy.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "k8s", value = KubernetesRunnerStrategy.class), + @JsonSubTypes.Type(name = "worker", value = WorkerRunnerStrategy.class), + @JsonSubTypes.Type(name = "taskType", value = TaskTypeRunnerStrategy.class) +}) +public interface RunnerStrategy +{ + String WORKER_NAME = "worker"; + + /** + * Enumerates the available runner types, each associated with a specific method of task execution. + * These runner types are used by the strategies to make decisions and by the system to route tasks appropriately. + */ + enum RunnerType + { + KUBERNETES_RUNNER_TYPE(KubernetesTaskRunnerFactory.TYPE_NAME), + WORKER_RUNNER_TYPE(WORKER_NAME); + + private final String type; + + RunnerType(String type) + { + this.type = type; + } + + public String getType() + { + return type; + } + } + + /** + * Analyzes the task and determines the appropriate runner type for executing it. + * + * @param task The task that needs to be executed. + * @return The runner type deemed most suitable for executing the task. + */ + RunnerType getRunnerTypeForTask(Task task); +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategy.java new file mode 100644 index 000000000000..6a16314be5b9 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategy.java @@ -0,0 +1,128 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.indexing.common.task.Task; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * Implementation of {@link RunnerStrategy} that allows dynamic selection of runner type based on task type. + * + *

This strategy checks each task's type against a set of overrides to determine the appropriate runner type. + * If no override is specified for a task's type, it uses a default runner. + * + *

Runner types are determined based on configurations provided at construction, including default runner + * type and specific overrides per task type. This strategy is designed for environments where tasks may require + * different execution environments (e.g., Kubernetes or worker nodes). + */ +public class TaskTypeRunnerStrategy implements RunnerStrategy +{ + @Nullable + private final Map overrides; + private final RunnerStrategy kubernetesRunnerStrategy = new KubernetesRunnerStrategy(); + private WorkerRunnerStrategy workerRunnerStrategy; + private final RunnerStrategy defaultRunnerStrategy; + private final String defaultRunner; + + @JsonCreator + public TaskTypeRunnerStrategy( + @JsonProperty("default") String defaultRunner, + @JsonProperty("overrides") @Nullable Map overrides + ) + { + Preconditions.checkNotNull(defaultRunner); + workerRunnerStrategy = new WorkerRunnerStrategy(); + defaultRunnerStrategy = RunnerType.WORKER_RUNNER_TYPE.getType().equals(defaultRunner) ? + workerRunnerStrategy : kubernetesRunnerStrategy; + validate(overrides); + this.defaultRunner = defaultRunner; + this.overrides = overrides; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getOverrides() + { + return overrides; + } + + @JsonProperty + public String getDefault() + { + return defaultRunner; + } + + @Override + public RunnerType getRunnerTypeForTask(Task task) + { + String runnerType = null; + if (overrides != null) { + runnerType = overrides.get(task.getType()); + } + + RunnerStrategy runnerStrategy = getRunnerSelectStrategy(runnerType); + return runnerStrategy.getRunnerTypeForTask(task); + } + + private RunnerStrategy getRunnerSelectStrategy(String runnerType) + { + if (runnerType == null) { + return defaultRunnerStrategy; + } + + if (WORKER_NAME.equals(runnerType)) { + return workerRunnerStrategy; + } else { + return kubernetesRunnerStrategy; + } + } + + private void validate(Map overrides) + { + if (overrides == null) { + return; + } + + boolean hasValidRunnerType = + overrides.values().stream().allMatch(v -> RunnerType.WORKER_RUNNER_TYPE.getType().equals(v) + || RunnerType.KUBERNETES_RUNNER_TYPE.getType().equals(v)); + Preconditions.checkArgument( + hasValidRunnerType, + "Invalid config in 'overrides'. Each runner type must be either '%s' or '%s'.", + RunnerType.WORKER_RUNNER_TYPE.getType(), + RunnerType.KUBERNETES_RUNNER_TYPE.getType() + ); + } + + @Override + public String toString() + { + return "TaskTypeRunnerStrategy{" + + "default=" + defaultRunner + + ", overrides=" + overrides + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategy.java new file mode 100644 index 000000000000..bd06f91aa8f1 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategy.java @@ -0,0 +1,43 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.indexing.common.task.Task; + +/** + * Implementation of {@link RunnerStrategy} that always selects the Worker runner type for tasks. + * + *

This strategy is specific for tasks that are intended to be executed in a Worker environment. + * Regardless of task specifics, this strategy always returns {@link RunnerType#WORKER_RUNNER_TYPE}. + */ +public class WorkerRunnerStrategy implements RunnerStrategy +{ + @JsonCreator + public WorkerRunnerStrategy() + { + } + + @Override + public RunnerType getRunnerTypeForTask(Task task) + { + return RunnerType.WORKER_RUNNER_TYPE; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfigTest.java index 979aba69291a..8ad631682f95 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerConfigTest.java @@ -20,8 +20,6 @@ package org.apache.druid.k8s.overlord; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; -import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -39,9 +37,8 @@ public void test_deserializable() throws IOException KubernetesAndWorkerTaskRunnerConfig.class ); - Assert.assertEquals(RemoteTaskRunnerFactory.TYPE_NAME, config.getWorkerTaskRunnerType()); - Assert.assertFalse(config.isSendAllTasksToWorkerTaskRunner()); - + Assert.assertEquals("worker", config.getRunnerStrategy()); + Assert.assertEquals("remote", config.getWorkerType()); } @Test @@ -49,7 +46,6 @@ public void test_withDefaults() { KubernetesAndWorkerTaskRunnerConfig config = new KubernetesAndWorkerTaskRunnerConfig(null, null); - Assert.assertEquals(HttpRemoteTaskRunnerFactory.TYPE_NAME, config.getWorkerTaskRunnerType()); - Assert.assertFalse(config.isSendAllTasksToWorkerTaskRunner()); + Assert.assertEquals(KubernetesTaskRunnerFactory.TYPE_NAME, config.getRunnerStrategy()); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactoryTest.java index c8e6d3afa03c..88696017d056 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactoryTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerFactoryTest.java @@ -22,6 +22,8 @@ import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; +import org.apache.druid.k8s.overlord.runnerstrategy.KubernetesRunnerStrategy; +import org.apache.druid.k8s.overlord.runnerstrategy.WorkerRunnerStrategy; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.EasyMockSupport; @@ -45,7 +47,8 @@ public void test_useHttpTaskRunner_asDefault() kubernetesTaskRunnerFactory, httpRemoteTaskRunnerFactory, remoteTaskRunnerFactory, - new KubernetesAndWorkerTaskRunnerConfig(null, null) + new KubernetesAndWorkerTaskRunnerConfig(null, null), + new WorkerRunnerStrategy() ); EasyMock.expect(httpRemoteTaskRunnerFactory.build()).andReturn(null); @@ -63,7 +66,8 @@ public void test_specifyRemoteTaskRunner() kubernetesTaskRunnerFactory, httpRemoteTaskRunnerFactory, remoteTaskRunnerFactory, - new KubernetesAndWorkerTaskRunnerConfig("remote", null) + new KubernetesAndWorkerTaskRunnerConfig(null, "remote"), + new WorkerRunnerStrategy() ); EasyMock.expect(remoteTaskRunnerFactory.build()).andReturn(null); @@ -81,7 +85,8 @@ public void test_specifyIncorrectTaskRunner_shouldThrowException() kubernetesTaskRunnerFactory, httpRemoteTaskRunnerFactory, remoteTaskRunnerFactory, - new KubernetesAndWorkerTaskRunnerConfig("noop", null) + new KubernetesAndWorkerTaskRunnerConfig(null, "noop"), + new KubernetesRunnerStrategy() ); EasyMock.expect(remoteTaskRunnerFactory.build()).andReturn(null); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerTest.java index 80010b9e5396..40ca6fc2f2b8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesAndWorkerTaskRunnerTest.java @@ -31,6 +31,9 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner; +import org.apache.druid.k8s.overlord.runnerstrategy.KubernetesRunnerStrategy; +import org.apache.druid.k8s.overlord.runnerstrategy.TaskTypeRunnerStrategy; +import org.apache.druid.k8s.overlord.runnerstrategy.WorkerRunnerStrategy; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.EasyMockSupport; @@ -67,7 +70,7 @@ public void setup() runner = new KubernetesAndWorkerTaskRunner( kubernetesTaskRunner, workerTaskRunner, - new KubernetesAndWorkerTaskRunnerConfig(null, null) + new KubernetesRunnerStrategy() ); } @@ -77,7 +80,7 @@ public void test_runOnKubernetes() throws ExecutionException, InterruptedExcepti KubernetesAndWorkerTaskRunner kubernetesAndWorkerTaskRunner = new KubernetesAndWorkerTaskRunner( kubernetesTaskRunner, workerTaskRunner, - new KubernetesAndWorkerTaskRunnerConfig(null, false) + new KubernetesRunnerStrategy() ); TaskStatus taskStatus = TaskStatus.success(ID); EasyMock.expect(kubernetesTaskRunner.run(task)).andReturn(Futures.immediateFuture(taskStatus)); @@ -93,7 +96,7 @@ public void test_runOnWorker() throws ExecutionException, InterruptedException KubernetesAndWorkerTaskRunner kubernetesAndWorkerTaskRunner = new KubernetesAndWorkerTaskRunner( kubernetesTaskRunner, workerTaskRunner, - new KubernetesAndWorkerTaskRunnerConfig(null, true) + new WorkerRunnerStrategy() ); TaskStatus taskStatus = TaskStatus.success(ID); EasyMock.expect(workerTaskRunner.run(task)).andReturn(Futures.immediateFuture(taskStatus)); @@ -103,6 +106,33 @@ public void test_runOnWorker() throws ExecutionException, InterruptedException verifyAll(); } + @Test + public void test_runOnKubernetesOrWorkerBasedOnStrategy() throws ExecutionException, InterruptedException + { + TaskTypeRunnerStrategy runnerStrategy = new TaskTypeRunnerStrategy("k8s", ImmutableMap.of("index_kafka", "worker")); + KubernetesAndWorkerTaskRunner kubernetesAndWorkerTaskRunner = new KubernetesAndWorkerTaskRunner( + kubernetesTaskRunner, + workerTaskRunner, + runnerStrategy + ); + Task taskMock = EasyMock.createMock(Task.class); + TaskStatus taskStatus = TaskStatus.success(ID); + EasyMock.expect(taskMock.getId()).andReturn(ID).anyTimes(); + + EasyMock.expect(taskMock.getType()).andReturn("index_kafka").once(); + EasyMock.expect(workerTaskRunner.run(taskMock)).andReturn(Futures.immediateFuture(taskStatus)).once(); + EasyMock.replay(taskMock, workerTaskRunner); + Assert.assertEquals(taskStatus, kubernetesAndWorkerTaskRunner.run(taskMock).get()); + EasyMock.verify(taskMock, workerTaskRunner); + EasyMock.reset(taskMock, workerTaskRunner); + + EasyMock.expect(taskMock.getType()).andReturn("compact").once(); + EasyMock.expect(kubernetesTaskRunner.run(taskMock)).andReturn(Futures.immediateFuture(taskStatus)).once(); + EasyMock.replay(taskMock, kubernetesTaskRunner); + Assert.assertEquals(taskStatus, kubernetesAndWorkerTaskRunner.run(taskMock).get()); + EasyMock.verify(taskMock, kubernetesTaskRunner); + } + @Test public void test_getUsedCapacity() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategyTest.java new file mode 100644 index 000000000000..880d5528ac7b --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/KubernetesRunnerStrategyTest.java @@ -0,0 +1,43 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import org.apache.druid.indexing.common.task.Task; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +@RunWith(EasyMockRunner.class) +public class KubernetesRunnerStrategyTest extends EasyMockSupport +{ + @Mock + Task task; + + @Test + public void test_kubernetesRunnerStrategy_returnsCorrectRunnerType() + { + KubernetesRunnerStrategy runnerStrategy = new KubernetesRunnerStrategy(); + + Assert.assertEquals(RunnerStrategy.RunnerType.KUBERNETES_RUNNER_TYPE, runnerStrategy.getRunnerTypeForTask(task)); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategyTest.java new file mode 100644 index 000000000000..a32630ed6144 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/TaskTypeRunnerStrategyTest.java @@ -0,0 +1,64 @@ +/* + * 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.k8s.overlord.runnerstrategy; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerFactory; +import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +@RunWith(EasyMockRunner.class) +public class TaskTypeRunnerStrategyTest extends EasyMockSupport +{ + @Mock + Task task; + + @Test + public void test_taskTypeRunnerStrategy_returnsCorrectRunnerType() + { + TaskTypeRunnerStrategy runnerStrategy = new TaskTypeRunnerStrategy("k8s", ImmutableMap.of("index_kafka", "worker")); + EasyMock.expect(task.getType()).andReturn("index_kafka"); + EasyMock.expectLastCall().once(); + EasyMock.expect(task.getType()).andReturn("compact"); + EasyMock.expectLastCall().once(); + replayAll(); + Assert.assertEquals(RunnerStrategy.WORKER_NAME, runnerStrategy.getRunnerTypeForTask(task).getType()); + Assert.assertEquals(KubernetesTaskRunnerFactory.TYPE_NAME, runnerStrategy.getRunnerTypeForTask(task).getType()); + verifyAll(); + } + + @Test(expected = IllegalArgumentException.class) + public void test_invalidOverridesConfig_shouldThrowException() + { + new TaskTypeRunnerStrategy( + "k8s", + ImmutableMap.of( + "index_kafka", + "non_exist_runner" + ) + ); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategyTest.java new file mode 100644 index 000000000000..1a3ae34fc6a7 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/runnerstrategy/WorkerRunnerStrategyTest.java @@ -0,0 +1,43 @@ +/* + * 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.k8s.overlord.runnerstrategy; + + +import org.apache.druid.indexing.common.task.Task; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +@RunWith(EasyMockRunner.class) +public class WorkerRunnerStrategyTest extends EasyMockSupport +{ + @Mock + Task task; + + @Test + public void test_workerRunnerStrategy_returnsCorrectRunnerType() + { + WorkerRunnerStrategy runnerStrategy = new WorkerRunnerStrategy(); + Assert.assertEquals(RunnerStrategy.WORKER_NAME, runnerStrategy.getRunnerTypeForTask(task).getType()); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesAndWorkerTaskRunnerConfig.json b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesAndWorkerTaskRunnerConfig.json index 757b07ebda52..43e7414f11f8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesAndWorkerTaskRunnerConfig.json +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesAndWorkerTaskRunnerConfig.json @@ -1,4 +1,4 @@ { - "workerTaskRunnerType": "remote", - "sendAllTasksToWorkerTaskRunner": false + "runnerStrategy.type": "worker", + "runnerStrategy.workerType": "remote" } \ No newline at end of file From f48263bbb3f1f258cf435b39ddf94a24ed32c741 Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Wed, 25 Oct 2023 22:37:30 +0200 Subject: [PATCH 02/15] Report function name for unknown exceptions during execution (#14987) * provide function name when unknown exceptions are encountered * fix keywords/etc * fix keywrod order - regex excercise * add test * add check&fix keywords * decoupledIgnore * Revert "decoupledIgnore" This reverts commit e922c820a7d563ca49c9c686644bed967c42cb4b. * unpatch Function * move to a different location * checkstyle --- .../druid/math/expr/FunctionalExpr.java | 11 +++++- .../apache/druid/math/expr/FunctionTest.java | 36 +++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java b/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java index b83efb2b2bcc..3d5a7f511f9e 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.vector.ExprVectorProcessor; @@ -186,7 +187,15 @@ public String toString() @Override public ExprEval eval(ObjectBinding bindings) { - return function.apply(args, bindings); + try { + return function.apply(args, bindings); + } + catch (DruidException | ExpressionValidationException e) { + throw e; + } + catch (Exception e) { + throw DruidException.defensive().build(e, "Invocation of function '%s' encountered exception.", name); + } } @Override diff --git a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java index a093db167bbe..d975ec069d55 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -23,10 +23,12 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.Expr.ObjectBinding; import org.apache.druid.segment.column.TypeStrategies; import org.apache.druid.segment.column.TypeStrategiesTest; import org.apache.druid.segment.column.TypeStrategy; @@ -38,6 +40,7 @@ import org.junit.Test; import javax.annotation.Nullable; + import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; @@ -46,6 +49,9 @@ import java.util.List; import java.util.Set; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + public class FunctionTest extends InitializedNullHandlingTest { private Expr.ObjectBinding bestEffortBindings; @@ -122,6 +128,36 @@ public void setup() allBindings = new Expr.ObjectBinding[]{bestEffortBindings, typedBindings}; } + @Test + public void testUnknownErrorsAreWrappedAndReported() + { + final Expr expr = Parser.parse("abs(x)", ExprMacroTable.nil()); + + ObjectBinding bind = new ObjectBinding() + { + + @Override + public ExpressionType getType(String name) + { + return ExpressionType.LONG_ARRAY; + } + + @Override + public Object get(String name) + { + throw new RuntimeException("nested-exception"); + } + }; + DruidException e = Assert.assertThrows(DruidException.class, + () -> { + expr.eval(bind); + }); + + assertEquals("Invocation of function 'abs' encountered exception.", e.getMessage()); + assertNotNull(e.getCause()); + assertEquals("nested-exception", e.getCause().getMessage()); + } + @Test public void testCaseSimple() { From e7b8e6569b05f9eca8e0a04ae7927adb3061d3f8 Mon Sep 17 00:00:00 2001 From: Pranav Date: Wed, 25 Oct 2023 21:49:58 -0700 Subject: [PATCH 03/15] Updating plugin which has fix for corrupt nodejs pkg (#15259) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 26f353534369..42ced7193ade 100644 --- a/pom.xml +++ b/pom.xml @@ -1809,7 +1809,7 @@ com.github.eirslett frontend-maven-plugin - 1.14.0 + 1.14.2 From fc0b940f785d64ede77cd2079ef30a78b76b7932 Mon Sep 17 00:00:00 2001 From: David Christle Date: Thu, 26 Oct 2023 14:51:01 -0700 Subject: [PATCH 04/15] Document the allowed range of announcer maxBytesPerNode (#15063) --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 4c0bb0c2b898..a8245f37acde 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -757,7 +757,7 @@ In current Druid, multiple data segments may be announced under the same Znode. |Property|Description|Default| |--------|-----------|-------| |`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50| -|`druid.announcer.maxBytesPerNode`|Max byte size for Znode.|524288| +|`druid.announcer.maxBytesPerNode`|Max byte size for Znode. Allowed range is [1024, 1048576].|524288| |`druid.announcer.skipDimensionsAndMetrics`|Skip Dimensions and Metrics list from segment announcements. NOTE: Enabling this will also remove the dimensions and metrics list from Coordinator and Broker endpoints.|false| |`druid.announcer.skipLoadSpec`|Skip segment LoadSpec from segment announcements. NOTE: Enabling this will also remove the loadspec from Coordinator and Broker endpoints.|false| From f1132d20c5756c35b26f5ad43008158c105dcd3f Mon Sep 17 00:00:00 2001 From: Alexander Saydakov <13126686+AlexanderSaydakov@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:28:33 -0700 Subject: [PATCH 05/15] use datasketches-java 4.2.0 (#15257) * use datasketches-java 4.2.0 * use exclusive mode * fixed issues raised by CodeQL * fixed issue raised by spotbugs * fixed issues raised by intellij * added missing import * Update QuantilesSketchKeyCollector search mode and adjust tests. * Update sizeOf functions and add unit tests * Add unit tests --------- Co-authored-by: AlexanderSaydakov Co-authored-by: Gian Merlino Co-authored-by: Adarsh Sanjeev --- .../KllDoublesSketchAggregatorFactory.java | 4 +- .../kll/KllFloatsSketchAggregatorFactory.java | 4 +- ...llDoublesSketchComplexMetricSerdeTest.java | 6 +- ...KllFloatsSketchComplexMetricSerdeTest.java | 4 +- .../QuantilesSketchKeyCollector.java | 3 +- .../QuantilesSketchKeyCollectorFactory.java | 60 +++++++++++-- .../apache/druid/msq/exec/MSQSelectTest.java | 28 +++--- .../SqlMSQStatementResourcePostTest.java | 25 ++++-- .../msq/statistics/ByteRowKeySerdeTest.java | 83 +++++++++++++++++ .../ArrayOfStringTuplesSerDe.java | 81 +++++++++++++++-- .../ArrayOfStringsNullSafeSerde.java | 12 ++- .../parallel/distribution/StringSketch.java | 3 +- .../ArrayOfStringTuplesSerDeTest.java | 89 +++++++++++++++++++ .../ArrayOfStringsNullSafeSerdeTest.java | 3 +- licenses.yaml | 2 +- pom.xml | 2 +- 16 files changed, 359 insertions(+), 50 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ByteRowKeySerdeTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDeTest.java diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java index 815227adf55b..267953e23647 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.kll.KllDoublesSketch; +import org.apache.datasketches.kll.KllSketch; +import org.apache.datasketches.kll.KllSketch.SketchType; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; import org.apache.druid.query.aggregation.AggregatorUtil; @@ -124,7 +126,7 @@ Class getSketchClass() @Override int getMaxSerializedSizeBytes(final int k, final long n) { - return KllDoublesSketch.getMaxSerializedSizeBytes(k, n, true); + return KllSketch.getMaxSerializedSizeBytes(k, n, SketchType.DOUBLES_SKETCH, true); } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java index 9cc61524615c..bdd672ab1257 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.kll.KllFloatsSketch; +import org.apache.datasketches.kll.KllSketch; +import org.apache.datasketches.kll.KllSketch.SketchType; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; import org.apache.druid.query.aggregation.AggregatorUtil; @@ -124,7 +126,7 @@ Class getSketchClass() @Override int getMaxSerializedSizeBytes(final int k, final long n) { - return KllFloatsSketch.getMaxSerializedSizeBytes(k, n, true); + return KllSketch.getMaxSerializedSizeBytes(k, n, SketchType.FLOATS_SKETCH, true); } @Override diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java index d0a263079906..730fb54c541d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java @@ -114,7 +114,7 @@ public void testSafeRead() objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray(); // corrupted sketch should fail with a regular java buffer exception, not all subsets actually fail with the same - // index out of bounds exceptions, but at least this many do + // sketches exceptions, but at least this many do for (int subset = 3; subset < 24; subset++) { final byte[] garbage2 = new byte[subset]; for (int i = 0; i < garbage2.length; i++) { @@ -123,7 +123,7 @@ public void testSafeRead() final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); Assert.assertThrows( - IndexOutOfBoundsException.class, + Exception.class, () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray() ); } @@ -132,7 +132,7 @@ public void testSafeRead() final byte[] garbage = new byte[]{0x01, 0x02}; final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); Assert.assertThrows( - IndexOutOfBoundsException.class, + Exception.class, () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray() ); } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java index 56a397789906..ee505fe65b88 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java @@ -123,7 +123,7 @@ public void testSafeRead() final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); Assert.assertThrows( - IndexOutOfBoundsException.class, + Exception.class, () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray() ); } @@ -132,7 +132,7 @@ public void testSafeRead() final byte[] garbage = new byte[]{0x01, 0x02}; final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); Assert.assertThrows( - IndexOutOfBoundsException.class, + Exception.class, () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray() ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java index 607265367c2c..a20ff40cc870 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java @@ -23,6 +23,7 @@ import com.google.common.primitives.Ints; import org.apache.datasketches.quantiles.ItemsSketch; import org.apache.datasketches.quantiles.ItemsUnion; +import org.apache.datasketches.quantilescommon.QuantileSearchCriteria; import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.RowKey; @@ -149,7 +150,7 @@ public ClusterByPartitions generatePartitionsWithTargetWeight(final long targetW final int numPartitions = Ints.checkedCast(LongMath.divide(sketch.getN(), targetWeight, RoundingMode.CEILING)); - final byte[][] quantiles = (sketch.getPartitionBoundaries(numPartitions)).boundaries; + final byte[][] quantiles = (sketch.getPartitionBoundaries(numPartitions, QuantileSearchCriteria.EXCLUSIVE)).boundaries; final List partitions = new ArrayList<>(); for (int i = 0; i < numPartitions; i++) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java index 3192813cfe1a..674dfe15acbb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.JsonDeserializer; import com.google.common.annotations.VisibleForTesting; import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.memory.WritableMemory; import org.apache.datasketches.quantiles.ItemsSketch; @@ -32,6 +33,7 @@ import java.io.IOException; import java.nio.ByteOrder; +import java.util.Arrays; import java.util.Comparator; public class QuantilesSketchKeyCollectorFactory @@ -91,9 +93,9 @@ public QuantilesSketchKeyCollector fromSnapshot(QuantilesSketchKeyCollectorSnaps return new QuantilesSketchKeyCollector(comparator, sketch, snapshot.getAverageKeyLength()); } - private static class ByteRowKeySerde extends ArrayOfItemsSerDe + static class ByteRowKeySerde extends ArrayOfItemsSerDe { - private static final ByteRowKeySerde INSTANCE = new ByteRowKeySerde(); + static final ByteRowKeySerde INSTANCE = new ByteRowKeySerde(); private ByteRowKeySerde() { @@ -126,22 +128,66 @@ public byte[] serializeToByteArray(final byte[][] items) } @Override - public byte[][] deserializeFromMemory(final Memory mem, final int numItems) + public byte[][] deserializeFromMemory(final Memory mem, long offsetBytes, final int numItems) { final byte[][] keys = new byte[numItems][]; - long keyPosition = (long) Integer.BYTES * numItems; + final long start = offsetBytes; + offsetBytes += (long) Integer.BYTES * numItems; for (int i = 0; i < numItems; i++) { - final int keyLength = mem.getInt((long) Integer.BYTES * i); + final int keyLength = mem.getInt(start + (long) Integer.BYTES * i); final byte[] keyBytes = new byte[keyLength]; - mem.getByteArray(keyPosition, keyBytes, 0, keyLength); + mem.getByteArray(offsetBytes, keyBytes, 0, keyLength); keys[i] = keyBytes; - keyPosition += keyLength; + offsetBytes += keyLength; } return keys; } + + @Override + public byte[] serializeToByteArray(final byte[] item) + { + final byte[] bytes = new byte[Integer.BYTES + item.length]; + ByteArrayUtil.putIntLE(bytes, 0, item.length); + ByteArrayUtil.copyBytes(item, 0, bytes, Integer.BYTES, item.length); + return bytes; + } + + @Override + public byte[][] deserializeFromMemory(final Memory mem, final int numItems) + { + return deserializeFromMemory(mem, 0, numItems); + } + + @Override + public int sizeOf(final byte[] item) + { + return Integer.BYTES + item.length; + } + + @Override + public int sizeOf(final Memory mem, long offsetBytes, final int numItems) + { + int length = Integer.BYTES * numItems; + for (int i = 0; i < numItems; i++) { + length += mem.getInt(offsetBytes + (long) Integer.BYTES * i); + } + return length; + } + + @Override + public String toString(final byte[] item) + { + return Arrays.toString(item); + } + + @Override + public Class getClassOfT() + { + return byte[].class; + } } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index b63ee479e202..441c98b91d8b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -226,7 +226,7 @@ public void testSelectOnFoo() .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with() - .rows(isPageSizeLimited() ? new long[]{1, 2, 3} : new long[]{6}) + .rows(isPageSizeLimited() ? new long[]{2, 2, 2} : new long[]{6}) .frames(isPageSizeLimited() ? new long[]{1, 1, 1} : new long[]{1}), 0, 0, "shuffle" ) @@ -290,7 +290,9 @@ public void testSelectOnFoo2() ) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(3).frames(1), + .with() + .rows(isPageSizeLimited() ? new long[]{1L, 2L} : new long[]{3L}) + .frames(isPageSizeLimited() ? new long[]{1L, 1L} : new long[]{1L}), 0, 0, "shuffle" ) .verifyResults(); @@ -353,7 +355,7 @@ public void testSelectOnFooDuplicateColumnNames() .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with() - .rows(isPageSizeLimited() ? new long[]{1, 2, 3} : new long[]{6}) + .rows(isPageSizeLimited() ? new long[]{2, 2, 2} : new long[]{6}) .frames(isPageSizeLimited() ? new long[]{1, 1, 1} : new long[]{1}), 0, 0, "shuffle" ) @@ -1442,8 +1444,8 @@ public void testExternSelectWithMultipleWorkers() throws IOException .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with() - .rows(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 2L} : new long[]{5L}) - .frames(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L} : new long[]{1L}), + .rows(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L, 1L} : new long[]{5L}) + .frames(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L, 1L} : new long[]{1L}), 0, 0, "shuffle" ) .setExpectedCountersForStageWorkerChannel( @@ -1459,27 +1461,27 @@ public void testExternSelectWithMultipleWorkers() throws IOException .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with() - .rows(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 2L} : new long[]{5L}) - .frames(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L} : new long[]{1L}), + .rows(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L, 1L} : new long[]{5L}) + .frames(isPageSizeLimited() ? new long[]{1L, 1L, 1L, 1L, 1L} : new long[]{1L}), 0, 1, "shuffle" ); // adding result stage counter checks if (isPageSizeLimited()) { selectTester.setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(2, 0, 2), + .with().rows(2, 0, 2, 0, 2), 1, 0, "input0" ).setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(2, 0, 2), + .with().rows(2, 0, 2, 0, 2), 1, 0, "output" ).setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(0, 2, 0, 4), + .with().rows(0, 2, 0, 2), 1, 1, "input0" ).setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(0, 2, 0, 4), + .with().rows(0, 2, 0, 2), 1, 1, "output" ); } @@ -1600,7 +1602,9 @@ public void testSelectOnUserDefinedSourceContainingWith() ) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher - .with().rows(3).frames(1), + .with() + .rows(isPageSizeLimited() ? new long[]{1, 2} : new long[]{3}) + .frames(isPageSizeLimited() ? new long[]{1, 1} : new long[]{1}), 0, 0, "shuffle" ) .verifyResults(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java index 6650c7785555..070b3ae46a71 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java @@ -325,9 +325,9 @@ public void testWithDurableStorage() throws IOException ).getEntity(); Assert.assertEquals(ImmutableList.of( - new PageInformation(0, 1L, 75L, 0, 0), - new PageInformation(1, 2L, 121L, 0, 1), - new PageInformation(2, 3L, 164L, 0, 2) + new PageInformation(0, 2L, 120L, 0, 0), + new PageInformation(1, 2L, 118L, 0, 1), + new PageInformation(2, 2L, 122L, 0, 2) ), sqlStatementResult.getResultSetInformation().getPages()); assertExpectedResults( @@ -348,7 +348,9 @@ public void testWithDurableStorage() throws IOException ); assertExpectedResults( - "{\"cnt\":1,\"dim1\":\"\"}\n\n", + "{\"cnt\":1,\"dim1\":\"\"}\n" + + "{\"cnt\":1,\"dim1\":\"10.1\"}\n" + + "\n", resource.doGetResults( sqlStatementResult.getQueryId(), 0L, @@ -359,8 +361,7 @@ public void testWithDurableStorage() throws IOException ); assertExpectedResults( - "{\"cnt\":1,\"dim1\":\"1\"}\n" - + "{\"cnt\":1,\"dim1\":\"def\"}\n" + "{\"cnt\":1,\"dim1\":\"def\"}\n" + "{\"cnt\":1,\"dim1\":\"abc\"}\n" + "\n", resource.doGetResults( @@ -412,7 +413,8 @@ public void testMultipleWorkersWithPageSizeLimiting() throws IOException new PageInformation(0, 2L, 128L, 0, 0), new PageInformation(1, 2L, 132L, 1, 1), new PageInformation(2, 2L, 128L, 0, 2), - new PageInformation(3, 4L, 228L, 1, 3) + new PageInformation(3, 2L, 132L, 1, 3), + new PageInformation(4, 2L, 130L, 0, 4) ), sqlStatementResult.getResultSetInformation().getPages()); @@ -457,12 +459,19 @@ public void testMultipleWorkersWithPageSizeLimiting() throws IOException SqlStatementResourceTest.makeOkRequest() ))); - Assert.assertEquals(rows.subList(6, 10), SqlStatementResourceTest.getResultRowsFromResponse(resource.doGetResults( + Assert.assertEquals(rows.subList(6, 8), SqlStatementResourceTest.getResultRowsFromResponse(resource.doGetResults( sqlStatementResult.getQueryId(), 3L, ResultFormat.ARRAY.name(), SqlStatementResourceTest.makeOkRequest() ))); + + Assert.assertEquals(rows.subList(8, 10), SqlStatementResourceTest.getResultRowsFromResponse(resource.doGetResults( + sqlStatementResult.getQueryId(), + 4L, + ResultFormat.ARRAY.name(), + SqlStatementResourceTest.makeOkRequest() + ))); } @Test diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ByteRowKeySerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ByteRowKeySerdeTest.java new file mode 100644 index 000000000000..9226ab4c81f0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ByteRowKeySerdeTest.java @@ -0,0 +1,83 @@ +/* + * 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.msq.statistics; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.key.KeyTestUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +public class ByteRowKeySerdeTest extends InitializedNullHandlingTest +{ + private final QuantilesSketchKeyCollectorFactory.ByteRowKeySerde serde = + QuantilesSketchKeyCollectorFactory.ByteRowKeySerde.INSTANCE; + + @Test + public void testByteArraySerde() + { + testSerde(new byte[]{1, 5, 9, 3}); + testSerde(new byte[][]{new byte[]{1, 5}, new byte[]{2, 3}, new byte[]{6, 7}}); + } + + @Test + public void testSerdeWithRowKeys() + { + RowSignature rowSignature = RowSignature.builder() + .add("x", ColumnType.LONG) + .add("y", ColumnType.LONG) + .build(); + + testSerde(KeyTestUtils.createKey(rowSignature, 2, 4).array()); + } + + @Test + public void testEmptyArray() + { + testSerde(new byte[][]{}); + testSerde(new byte[][]{new byte[]{1, 5}, new byte[]{}, new byte[]{2, 3}}); + } + + private void testSerde(byte[] byteRowKey) + { + byte[] bytes = serde.serializeToByteArray(byteRowKey); + Assert.assertEquals(serde.sizeOf(byteRowKey), bytes.length); + + Memory wrappedMemory = Memory.wrap(bytes); + Assert.assertEquals(serde.sizeOf(wrappedMemory, 0, 1), bytes.length); + + byte[][] deserialized = serde.deserializeFromMemory(wrappedMemory, 1); + Assert.assertArrayEquals(new byte[][]{byteRowKey}, deserialized); + } + + private void testSerde(byte[][] inputArray) + { + byte[] bytes = serde.serializeToByteArray(inputArray); + Assert.assertEquals(serde.sizeOf(inputArray), bytes.length); + + Memory wrappedMemory = Memory.wrap(bytes); + Assert.assertEquals(serde.sizeOf(wrappedMemory, 0, inputArray.length), bytes.length); + + byte[][] deserialized = serde.deserializeFromMemory(wrappedMemory, inputArray.length); + Assert.assertArrayEquals(inputArray, deserialized); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDe.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDe.java index e3f76b5b92fd..8bba9f65aafd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDe.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDe.java @@ -21,8 +21,9 @@ import org.apache.datasketches.common.ArrayOfItemsSerDe; import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; import org.apache.datasketches.memory.Memory; -import org.apache.datasketches.memory.WritableMemory; import org.apache.datasketches.memory.internal.UnsafeUtil; import org.apache.druid.data.input.StringTuple; @@ -36,7 +37,7 @@ public class ArrayOfStringTuplesSerDe extends ArrayOfItemsSerDe private static final ArrayOfStringsNullSafeSerde STRINGS_SERDE = new ArrayOfStringsNullSafeSerde(); @Override - public byte[] serializeToByteArray(StringTuple[] items) + public byte[] serializeToByteArray(final StringTuple[] items) { int length = 0; final byte[][] itemsBytes = new byte[items.length][]; @@ -49,29 +50,27 @@ public byte[] serializeToByteArray(StringTuple[] items) } final byte[] bytes = new byte[length]; - final WritableMemory mem = WritableMemory.writableWrap(bytes); - long offsetBytes = 0; + int offsetBytes = 0; for (int i = 0; i < items.length; i++) { // Add the number of items in the StringTuple - mem.putInt(offsetBytes, items[i].size()); + ByteArrayUtil.putIntLE(bytes, offsetBytes, items[i].size()); offsetBytes += Integer.BYTES; // Add the size of byte content for the StringTuple - mem.putInt(offsetBytes, itemsBytes[i].length); + ByteArrayUtil.putIntLE(bytes, offsetBytes, itemsBytes[i].length); offsetBytes += Integer.BYTES; // Add the byte contents of the StringTuple - mem.putByteArray(offsetBytes, itemsBytes[i], 0, itemsBytes[i].length); + ByteArrayUtil.copyBytes(itemsBytes[i], 0, bytes, offsetBytes, itemsBytes[i].length); offsetBytes += itemsBytes[i].length; } return bytes; } @Override - public StringTuple[] deserializeFromMemory(Memory mem, int numItems) + public StringTuple[] deserializeFromMemory(final Memory mem, long offsetBytes, final int numItems) { final StringTuple[] array = new StringTuple[numItems]; - long offsetBytes = 0; for (int i = 0; i < numItems; i++) { // Read the number of items in the StringTuple UnsafeUtil.checkBounds(offsetBytes, Integer.BYTES, mem.getCapacity()); @@ -96,4 +95,68 @@ public StringTuple[] deserializeFromMemory(Memory mem, int numItems) } return array; } + + @Override + public byte[] serializeToByteArray(final StringTuple item) + { + final byte[] itemBytes = STRINGS_SERDE.serializeToByteArray(item.toArray()); + final byte[] bytes = new byte[Integer.BYTES * 2 + itemBytes.length]; + int offsetBytes = 0; + ByteArrayUtil.putIntLE(bytes, offsetBytes, item.size()); + offsetBytes += Integer.BYTES; + ByteArrayUtil.putIntLE(bytes, offsetBytes, itemBytes.length); + offsetBytes += Integer.BYTES; + ByteArrayUtil.copyBytes(itemBytes, 0, bytes, offsetBytes, itemBytes.length); + return bytes; + } + + @Override + public StringTuple[] deserializeFromMemory(final Memory mem, final int numItems) + { + return deserializeFromMemory(mem, 0, numItems); + } + + @Override + public int sizeOf(final StringTuple item) + { + // Two integers to store number of strings in the tuple and the size of the byte array + int length = 2 * Integer.BYTES; + for (final String s : item.toArray()) { + length += STRINGS_SERDE.sizeOf(s); + } + return length; + } + + @Override + public int sizeOf(final Memory mem, long offsetBytes, final int numItems) + { + final long start = offsetBytes; + for (int i = 0; i < numItems; i++) { + // Skip the number of items in the StringTuple + Util.checkBounds(offsetBytes, Integer.BYTES, mem.getCapacity()); + offsetBytes += Integer.BYTES; + + // Read the size of byte content + Util.checkBounds(offsetBytes, Integer.BYTES, mem.getCapacity()); + final int byteContentSize = mem.getInt(offsetBytes); + offsetBytes += Integer.BYTES; + + // Skip the byte content + Util.checkBounds(offsetBytes, byteContentSize, mem.getCapacity()); + offsetBytes += byteContentSize; + } + return (int) (offsetBytes - start); + } + + @Override + public String toString(final StringTuple item) + { + return item.toString(); + } + + @Override + public Class getClassOfT() + { + return StringTuple.class; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerde.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerde.java index b5a8393b1724..cd1057417170 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerde.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerde.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel.distribution; -import org.apache.datasketches.common.ArrayOfItemsSerDe; import org.apache.datasketches.common.ArrayOfStringsSerDe; import org.apache.datasketches.common.Util; import org.apache.datasketches.memory.Memory; @@ -35,7 +34,7 @@ * The implementation is the same as {@link ArrayOfStringsSerDe}, except this * class handles null String values as well. */ -public class ArrayOfStringsNullSafeSerde extends ArrayOfItemsSerDe +public class ArrayOfStringsNullSafeSerde extends ArrayOfStringsSerDe { private static final int NULL_STRING_LENGTH = -1; @@ -106,5 +105,14 @@ public String[] deserializeFromMemory(final Memory mem, final int numItems) return array; } + @Override + public int sizeOf(String item) + { + if (item == null) { + return Integer.BYTES; + } else { + return super.sizeOf(item); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/StringSketch.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/StringSketch.java index e47fcf78c135..e2794f6b1caf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/StringSketch.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/StringSketch.java @@ -33,6 +33,7 @@ import com.google.common.base.Preconditions; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.quantiles.ItemsSketch; +import org.apache.datasketches.quantilescommon.QuantileSearchCriteria; import org.apache.druid.data.input.StringTuple; import org.apache.druid.timeline.partition.PartitionBoundaries; @@ -128,7 +129,7 @@ private PartitionBoundaries getEvenPartitionsByCount(int evenPartitionCount) if (delegate.isEmpty()) { return new PartitionBoundaries(new StringTuple[0]); } - return new PartitionBoundaries((delegate.getPartitionBoundaries(evenPartitionCount)).boundaries); + return new PartitionBoundaries((delegate.getPartitionBoundaries(evenPartitionCount, QuantileSearchCriteria.EXCLUSIVE)).boundaries); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDeTest.java new file mode 100644 index 000000000000..9b909948ad04 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringTuplesSerDeTest.java @@ -0,0 +1,89 @@ +/* + * 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.indexing.common.task.batch.parallel.distribution; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.data.input.StringTuple; +import org.junit.Assert; +import org.junit.Test; + +public class ArrayOfStringTuplesSerDeTest +{ + + private final ArrayOfStringTuplesSerDe serde = new ArrayOfStringTuplesSerDe(); + + @Test + public void testStringTupleSerde() + { + testSerde(StringTuple.create("abc")); + testSerde(StringTuple.create("abc", "def", "xyz")); + testSerde(new StringTuple[]{StringTuple.create("abc"), StringTuple.create("def", "efg"), StringTuple.create("z")}); + } + + @Test + public void testEmptyTuple() + { + testSerde(StringTuple.create()); + testSerde(new StringTuple[]{}); + } + + @Test + public void testArrayWithNullAndEmptyString() + { + testSerde(StringTuple.create("")); + testSerde(StringTuple.create("abc", "def", "")); + testSerde(StringTuple.create("abc", null, "def")); + testSerde(new StringTuple[]{StringTuple.create(null, null), StringTuple.create(null, null)}); + testSerde(new StringTuple[]{StringTuple.create("", ""), StringTuple.create("")}); + testSerde(StringTuple.create("", null, "abc")); + } + + @Test + public void testSizeOf() + { + StringTuple stringTuple = StringTuple.create("a", "b"); + Assert.assertEquals(serde.sizeOf(stringTuple), serde.serializeToByteArray(stringTuple).length); + } + + private void testSerde(StringTuple stringTuple) + { + byte[] bytes = serde.serializeToByteArray(stringTuple); + Assert.assertEquals(serde.sizeOf(stringTuple), bytes.length); + + Memory wrappedMemory = Memory.wrap(bytes); + Assert.assertEquals(serde.sizeOf(wrappedMemory, 0, 1), bytes.length); + + StringTuple[] deserialized = serde.deserializeFromMemory(wrappedMemory, 1); + Assert.assertArrayEquals(new StringTuple[]{stringTuple}, deserialized); + } + + private void testSerde(StringTuple[] inputArray) + { + byte[] bytes = serde.serializeToByteArray(inputArray); + Assert.assertEquals(serde.sizeOf(inputArray), bytes.length); + + Memory wrappedMemory = Memory.wrap(bytes); + Assert.assertEquals(serde.sizeOf(wrappedMemory, 0, inputArray.length), bytes.length); + + StringTuple[] deserialized = serde.deserializeFromMemory(wrappedMemory, inputArray.length); + Assert.assertArrayEquals(inputArray, deserialized); + } + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerdeTest.java index 927f311e4f95..bce82ee0dbc0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/distribution/ArrayOfStringsNullSafeSerdeTest.java @@ -84,8 +84,9 @@ public void testIllegalStrLength() private void testSerde(String... inputArray) { byte[] bytes = serde.serializeToByteArray(inputArray); + Assert.assertEquals(serde.sizeOf(inputArray), bytes.length); String[] deserialized = serde.deserializeFromMemory(Memory.wrap(bytes), inputArray.length); - Assert.assertEquals(inputArray, deserialized); + Assert.assertArrayEquals(inputArray, deserialized); } } diff --git a/licenses.yaml b/licenses.yaml index a207fb1b0acf..eb2d63dd2456 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -3477,7 +3477,7 @@ name: DataSketches license_category: binary module: java-core license_name: Apache License version 2.0 -version: 4.1.0 +version: 4.2.0 libraries: - org.apache.datasketches: datasketches-java diff --git a/pom.xml b/pom.xml index 42ced7193ade..cd0d740c72fc 100644 --- a/pom.xml +++ b/pom.xml @@ -86,7 +86,7 @@ default_config.fmpp --> 1.35.0 - 4.1.0 + 4.2.0 2.2.0 10.14.2.0 4.2.19 From e9b7e4a0eb227c203419d3f05a7040b8dc0d2e3a Mon Sep 17 00:00:00 2001 From: Simon Hofbauer <61789910+hofi1@users.noreply.github.com> Date: Thu, 26 Oct 2023 22:27:09 -0500 Subject: [PATCH 06/15] fix JSON flaky tests (#15261) Co-authored-by: simonh5 --- .../java/util/emitter/core/EmitterTest.java | 38 ++++++++++++------- .../core/ParametrizedUriEmitterTest.java | 4 +- 2 files changed, 27 insertions(+), 15 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/EmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/EmitterTest.java index 39b55596ca00..62244d3bd288 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/EmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/EmitterTest.java @@ -240,12 +240,14 @@ protected ListenableFuture go(Request request) throws JsonProcessingEx request.getHeaders().get(HttpHeaders.Names.CONTENT_TYPE) ); Assert.assertEquals( - StringUtils.format( + JSON_MAPPER.readTree(StringUtils.format( "[%s,%s]\n", JSON_MAPPER.writeValueAsString(events.get(0)), JSON_MAPPER.writeValueAsString(events.get(1)) - ), - StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + )), + JSON_MAPPER.readTree( + StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + ) ); return GoHandlers.immediateFuture(okResponse()); @@ -282,12 +284,14 @@ protected ListenableFuture go(Request request) throws JsonProcessingEx request.getHeaders().get(HttpHeaders.Names.CONTENT_TYPE) ); Assert.assertEquals( - StringUtils.format( + JSON_MAPPER.readTree(StringUtils.format( "[%s,%s]\n", JSON_MAPPER.writeValueAsString(events.get(0)), JSON_MAPPER.writeValueAsString(events.get(1)) - ), - StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + )), + JSON_MAPPER.readTree( + StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + ) ); return GoHandlers.immediateFuture(okResponse()); @@ -467,12 +471,14 @@ protected ListenableFuture go(Request request) throws JsonProcessingEx request.getHeaders().get(HttpHeaders.Names.AUTHORIZATION) ); Assert.assertEquals( - StringUtils.format( + JSON_MAPPER.readTree(StringUtils.format( "%s\n%s\n", JSON_MAPPER.writeValueAsString(events.get(0)), JSON_MAPPER.writeValueAsString(events.get(1)) - ), - StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + )), + JSON_MAPPER.readTree( + StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + ) ); return GoHandlers.immediateFuture(okResponse()); @@ -521,12 +527,14 @@ protected ListenableFuture go(Request request) throws JsonProcessingEx request.getHeaders().get(HttpHeaders.Names.CONTENT_TYPE) ); Assert.assertEquals( - StringUtils.format( + JSON_MAPPER.readTree(StringUtils.format( "[%s,%s]\n", JSON_MAPPER.writeValueAsString(events.get(counter.getAndIncrement())), JSON_MAPPER.writeValueAsString(events.get(counter.getAndIncrement())) - ), - StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + )), + JSON_MAPPER.readTree( + StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString() + ) ); return GoHandlers.immediateFuture(okResponse()); @@ -584,12 +592,14 @@ protected ListenableFuture go(Request request) throws IOException CompressionUtils.gunzip(new ByteArrayInputStream(dataArray), baos); Assert.assertEquals( - StringUtils.format( + JSON_MAPPER.readTree(StringUtils.format( "[%s,%s]\n", JSON_MAPPER.writeValueAsString(events.get(0)), JSON_MAPPER.writeValueAsString(events.get(1)) - ), + )), + JSON_MAPPER.readTree( baos.toString(StandardCharsets.UTF_8.name()) + ) ); return GoHandlers.immediateFuture(okResponse()); diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java index f7d7a8337a20..c5dc26432fe0 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java @@ -155,7 +155,9 @@ protected ListenableFuture go(Request request) "http://example.com/test1", StringUtils.format("[%s]\n", JSON_MAPPER.writeValueAsString(events.get(0))), "http://example.com/test2", StringUtils.format("[%s]\n", JSON_MAPPER.writeValueAsString(events.get(1))) ); - Assert.assertEquals(expected, results); + for (Map.Entry entry : expected.entrySet()) { + Assert.assertEquals(JSON_MAPPER.readTree(expected.get(entry.getKey())), JSON_MAPPER.readTree(results.get(entry.getKey()))); + } } @Test From 7c8e841362792b1bfc45d8b8d12857f7cad38526 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 27 Oct 2023 09:29:18 +0530 Subject: [PATCH 07/15] Suppress CVE's in master (#15231) --- distribution/bin/check-licenses.py | 2 ++ owasp-dependency-check-suppressions.xml | 34 ++++++++++++++++++++++--- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/distribution/bin/check-licenses.py b/distribution/bin/check-licenses.py index ae07e5a03733..b069d9545b36 100755 --- a/distribution/bin/check-licenses.py +++ b/distribution/bin/check-licenses.py @@ -266,6 +266,8 @@ def build_compatible_license_names(): compatible_licenses['Eclipse Public License - Version 1.0'] = 'Eclipse Public License 1.0' compatible_licenses['Eclipse Public License, Version 1.0'] = 'Eclipse Public License 1.0' compatible_licenses['Eclipse Public License v1.0'] = 'Eclipse Public License 1.0' + compatible_licenses['Eclipse Public License - v1.0'] = 'Eclipse Public License 1.0' + compatible_licenses['Eclipse Public License - v 1.0'] = 'Eclipse Public License 1.0' compatible_licenses['EPL 1.0'] = 'Eclipse Public License 1.0' compatible_licenses['Eclipse Public License 2.0'] = 'Eclipse Public License 2.0' diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index e33231ea9ee3..ab6e6176994f 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -759,6 +759,7 @@ CVE-2023-1370 CVE-2023-37475 CVE-2023-39410 + CVE-2023-44487 @@ -766,6 +767,7 @@ file name: hadoop-client-api-3.3.6.jar: jquery.dataTables.min.js (pkg:javascript/jquery.datatables@1.10.18) ]]> prototype pollution + CVE-2020-28458 - ^pkg:maven/org\.codehaus\.plexus/plexus-interpolation@.*$ CVE-2022-4244 + + + + CVE-2023-5072 + + + + + + CVE-2023-44981 + + + + + + CVE-2023-4586 + From 60c2ad597ab44610298e94987582443368863269 Mon Sep 17 00:00:00 2001 From: kaisun2000 <52840222+kaisun2000@users.noreply.github.com> Date: Fri, 27 Oct 2023 01:50:19 -0700 Subject: [PATCH 08/15] Enhance json parser error logging to better track Istio Proxy error message (#15176) Currently the inter Druid communication via rest endpoints is based on json formatted payload. Upon parsing error, there is only a generic exception stating expected json token type and current json token type. There is no detailed error log about the content of the payload causing the violation. In the micro-service world, the trend is to deploy the Druid servers in k8 with the mesh network. Often the istio proxy or other proxies is used to intercept the network connection between Druid servers. The proxy may give error messages for various reasons. These error messages are not expected by the json parser. The generic error message from Druid can be very misleading as the user may think the message is based on the response from the other Druid server. For example, this is an example of mysterious error message QueryInterruptedException{msg=Next token wasn't a START_ARRAY, was[VALUE_STRING] from url[http://xxxxx:8088/druid/v2/], code=Unknown exception, class=org.apache.druid.java.util.common.IAE, host=xxxxx:8088}" While the context of the message is the following from the proxy when it can't tunnel the network connection. pstream connect error or disconnect/reset before header So this very simple PR is just to enhance the logging and get the real underlying message printed out. This would save a lot of head scratching time if Druid is deployed with mesh network. Co-authored-by: Kai Sun --- .../druid/client/JsonParserIterator.java | 11 +++- .../druid/client/JsonParserIteratorTest.java | 52 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java index 237614fb4ffa..00dea3dff0e0 100644 --- a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java +++ b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java @@ -185,8 +185,17 @@ private void init() } else if (nextToken == JsonToken.START_OBJECT) { throw convertException(jp.getCodec().readValue(jp, QueryException.class)); } else { + String errMsg = jp.getValueAsString(); + if (errMsg != null) { + errMsg = errMsg.substring(0, Math.min(errMsg.length(), 192)); + } throw convertException( - new IAE("Next token wasn't a START_ARRAY, was[%s] from url[%s]", jp.getCurrentToken(), url) + new IAE( + "Next token wasn't a START_ARRAY, was[%s] from url[%s] with value[%s]", + jp.getCurrentToken(), + url, + errMsg + ) ); } } diff --git a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java index 3ac2b4122d29..5cce3c2fd3f3 100644 --- a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java +++ b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java @@ -316,8 +316,60 @@ private Query mockQuery(String queryId, long timeoutAt) } } + public static class IAEExceptionConversionTest + { + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private String errorMessage = "pstream connect error or disconnect/reset before header"; + private String nullErrMsg = null; + + @Test + public void testNullErrorMsg() throws JsonProcessingException + { + JsonParserIterator iterator = new JsonParserIterator<>( + JAVA_TYPE, + Futures.immediateFuture( + mockErrorResponse(nullErrMsg) + ), + URL, + null, + HOST, + OBJECT_MAPPER + ); + + expectedException.expect(QueryInterruptedException.class); + expectedException.expectMessage(""); + iterator.hasNext(); + } + + @Test + public void testParsingError() throws JsonProcessingException + { + JsonParserIterator iterator = new JsonParserIterator<>( + JAVA_TYPE, + Futures.immediateFuture( + mockErrorResponse(errorMessage) + ), + URL, + null, + HOST, + OBJECT_MAPPER + ); + + expectedException.expect(QueryInterruptedException.class); + expectedException.expectMessage(errorMessage); + iterator.hasNext(); + } + } + private static InputStream mockErrorResponse(Exception e) throws JsonProcessingException { return new ByteArrayInputStream(OBJECT_MAPPER.writeValueAsBytes(e)); } + + private static InputStream mockErrorResponse(String errMsg) throws JsonProcessingException + { + return new ByteArrayInputStream(OBJECT_MAPPER.writeValueAsBytes(errMsg)); + } } From 737947754dfed23e649d980e2fc71b33ecb6e479 Mon Sep 17 00:00:00 2001 From: 317brian <53799971+317brian@users.noreply.github.com> Date: Fri, 27 Oct 2023 10:29:34 -0700 Subject: [PATCH 09/15] docs: add concurent compaction docs (#15218) Co-authored-by: Kashif Faraz --- docs/data-management/automatic-compaction.md | 102 ++++++++++- docs/data-management/compaction.md | 146 +--------------- docs/data-management/manual-compaction.md | 167 +++++++++++++++++++ docs/ingestion/ingestion-spec.md | 2 +- website/sidebars.json | 14 +- 5 files changed, 290 insertions(+), 141 deletions(-) create mode 100644 docs/data-management/manual-compaction.md diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md index 8d696a86d4ef..4de4f1f3763b 100644 --- a/docs/data-management/automatic-compaction.md +++ b/docs/data-management/automatic-compaction.md @@ -162,7 +162,7 @@ To get statistics by API, send a [`GET` request](../api-reference/automatic-comp ## Examples -The following examples demonstrate potential use cases in which auto-compaction may improve your Druid performance. See more details in [Compaction strategies](../data-management/compaction.md#compaction-strategies). The examples in this section do not change the underlying data. +The following examples demonstrate potential use cases in which auto-compaction may improve your Druid performance. See more details in [Compaction strategies](../data-management/compaction.md#compaction-guidelines). The examples in this section do not change the underlying data. ### Change segment granularity @@ -203,6 +203,106 @@ The following auto-compaction configuration compacts updates the `wikipedia` seg } ``` +## Concurrent append and replace + +:::info +Concurrent append and replace is an [experimental feature](../development/experimental.md) and is not currently available for SQL-based ingestion. +::: + +This feature allows you to safely replace the existing data in an interval of a datasource while new data is being appended to that interval. One of the most common applications of this is appending new data (using say streaming ingestion) to an interval while compaction of that interval is already in progress. + +To set up concurrent append and replace, you need to ensure that your ingestion jobs have the appropriate lock types: + +You can enable concurrent append and replace by ensuring the following: +- The append task (with `appendToExisting` set to `true`) has `taskLockType` set to `APPEND` in the task context. +- The replace task (with `appendToExisting` set to `false`) has `taskLockType` set to `REPLACE` in the task context. +- The segment granularity of the append task is equal to or finer than the segment granularity of the replace task. + +:::info + +When using concurrent append and replace, keep the following in mind: + +- Concurrent append and replace fails if the task with `APPEND` lock uses a coarser segment granularity than the task with the `REPLACE` lock. For example, if the `APPEND` task uses a segment granularity of YEAR and the `REPLACE` task uses a segment granularity of MONTH, you should not use concurrent append and replace. + +- Only a single task can hold a `REPLACE` lock on a given interval of a datasource. + +- Multiple tasks can hold `APPEND` locks on a given interval of a datasource and append data to that interval simultaneously. + +::: + + +### Configure concurrent append and replace + +##### Update the compaction settings with the API + + Prepare your datasource for concurrent append and replace by setting its task lock type to `REPLACE`. +Add the `taskContext` like you would any other automatic compaction setting through the API: + +```shell +curl --location --request POST 'http://localhost:8081/druid/coordinator/v1/config/compaction' \ +--header 'Content-Type: application/json' \ +--data-raw '{ + "dataSource": "YOUR_DATASOURCE", + "taskContext": { + "taskLockType": "REPLACE" + } +}' +``` + +##### Update the compaction settings with the UI + +In the **Compaction config** for a datasource, set **Allow concurrent compactions (experimental)** to **True**. + +#### Add a task lock type to your ingestion job + +Next, you need to configure the task lock type for your ingestion job: + +- For streaming jobs, the context parameter goes in your supervisor spec, and the lock type is always `APPEND` +- For legacy JSON-based batch ingestion, the context parameter goes in your ingestion spec, and the lock type can be either `APPEND` or `REPLACE`. + +You can provide the context parameter through the API like any other parameter for ingestion job or through the UI. + +##### Add the task lock type through the API + +Add the following JSON snippet to your supervisor or ingestion spec if you're using the API: + +```json +"context": { + "taskLockType": LOCK_TYPE +} +``` + +The `LOCK_TYPE` depends on what you're trying to accomplish. + +Set `taskLockType` to `APPEND` if either of the following are true: + +- Dynamic partitioning with append to existing is set to `true` +- The ingestion job is a streaming ingestion job + +If you have multiple ingestion jobs that append all targeting the same datasource and want them to run simultaneously, you need to also include the following context parameter: + +```json +"useSharedLock": "true" +``` + +Keep in mind that `taskLockType` takes precedence over `useSharedLock`. Do not use it with `REPLACE` task locks. + + +Set `taskLockType` to `REPLACE` if you're replacing data. For example, if you use any of the following partitioning types, use `REPLACE`: + +- hash partitioning +- range partitioning +- dynamic partitioning with append to existing set to `false` + + +##### Add a task lock using the Druid console + +As part of the **Load data** wizard for classic batch (JSON-based ingestion) and streaming ingestion, you can configure the task lock type for the ingestion during the **Publish** step: + +- If you set **Append to existing** to **True**, you can then set **Allow concurrent append tasks (experimental)** to **True**. +- If you set **Append to existing** to **False**, you can then set **Allow concurrent replace tasks (experimental)** to **True**. + + ## Learn more See the following topics for more information: diff --git a/docs/data-management/compaction.md b/docs/data-management/compaction.md index c166623e887d..b1daf275d9c3 100644 --- a/docs/data-management/compaction.md +++ b/docs/data-management/compaction.md @@ -22,9 +22,10 @@ description: "Defines compaction and automatic compaction (auto-compaction or au ~ specific language governing permissions and limitations ~ under the License. --> + Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. In some cases the compacted segments are larger, but there are fewer of them. In other cases the compacted segments may be smaller. Compaction tends to increase performance because optimized segments require less per-segment processing and less memory overhead for ingestion and for querying paths. -## Compaction strategies +## Compaction guidelines There are several cases to consider compaction for segment optimization: @@ -43,18 +44,20 @@ By default, compaction does not modify the underlying data of the segments. Howe Compaction does not improve performance in all situations. For example, if you rewrite your data with each ingestion task, you don't need to use compaction. See [Segment optimization](../operations/segment-optimization.md) for additional guidance to determine if compaction will help in your environment. -## Types of compaction +## Ways to run compaction -You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using its [segment search policy](../design/coordinator.md#segment-search-policy-in-automatic-compaction), the Coordinator periodically identifies segments for compaction starting from newest to oldest. When the Coordinator discovers segments that have not been compacted or segments that were compacted with a different or changed spec, it submits compaction tasks for the time interval covering those segments. +Automatic compaction, also called auto-compaction, works in most use cases and should be your first option. -Automatic compaction works in most use cases and should be your first option. To learn more, see [Automatic compaction](../data-management/automatic-compaction.md). +The Coordinator uses its [segment search policy](../design/coordinator.md#segment-search-policy-in-automatic-compaction) to periodically identify segments for compaction starting from newest to oldest. When the Coordinator discovers segments that have not been compacted or segments that were compacted with a different or changed spec, it submits compaction tasks for the time interval covering those segments. + +To learn more, see [Automatic compaction](../data-management/automatic-compaction.md). In cases where you require more control over compaction, you can manually submit compaction tasks. For example: - Automatic compaction is running into the limit of task slots available to it, so tasks are waiting for previous automatic compaction tasks to complete. Manual compaction can use all available task slots, therefore you can complete compaction more quickly by submitting more concurrent tasks for more intervals. - You want to force compaction for a specific time range or you want to compact data out of chronological order. -See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks. +See [Setting up a manual compaction task](./manual-compaction.md#setting-up-manual-compaction) for more about manual compaction tasks. ## Data handling with compaction @@ -101,141 +104,10 @@ Druid only rolls up the output segment when `rollup` is set for all input segmen See [Roll-up](../ingestion/rollup.md) for more details. You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes). -## Setting up manual compaction - -To perform a manual compaction, you submit a compaction task. Compaction tasks merge all segments for the defined interval according to the following syntax: - -```json -{ - "type": "compact", - "id": , - "dataSource": , - "ioConfig": , - "dimensionsSpec": , - "transformSpec": , - "metricsSpec": , - "tuningConfig": , - "granularitySpec": , - "context": -} -``` - -|Field|Description|Required| -|-----|-----------|--------| -|`type`|Task type. Set the value to `compact`.|Yes| -|`id`|Task ID|No| -|`dataSource`|Data source name to compact|Yes| -|`ioConfig`|I/O configuration for compaction task. See [Compaction I/O configuration](#compaction-io-configuration) for details.|Yes| -|`dimensionsSpec`|When set, the compaction task uses the specified `dimensionsSpec` rather than generating one from existing segments. See [Compaction dimensionsSpec](#compaction-dimensions-spec) for details.|No| -|`transformSpec`|When set, the compaction task uses the specified `transformSpec` rather than using `null`. See [Compaction transformSpec](#compaction-transform-spec) for details.|No| -|`metricsSpec`|When set, the compaction task uses the specified `metricsSpec` rather than generating one from existing segments.|No| -|`segmentGranularity`|Deprecated. Use `granularitySpec`.|No| -|`tuningConfig`|[Tuning configuration](../ingestion/native-batch.md#tuningconfig) for parallel indexing. `awaitSegmentAvailabilityTimeoutMillis` value is not supported for compaction tasks. Leave this parameter at the default value, 0.|No| -|`granularitySpec`|When set, the compaction task uses the specified `granularitySpec` rather than generating one from existing segments. See [Compaction `granularitySpec`](#compaction-granularity-spec) for details.|No| -|`context`|[Task context](../ingestion/tasks.md#context)|No| - -:::info - Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails. -::: - -To control the number of result segments per time chunk, you can set [`maxRowsPerSegment`](../ingestion/native-batch.md#partitionsspec) or [`numShards`](../ingestion/../ingestion/native-batch.md#tuningconfig). - -:::info - You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals. -::: - -A compaction task internally generates an `index` or `index_parallel` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [`druid` input source](../ingestion/input-sources.md), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default. - -Compaction tasks typically fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to maximize performance and minimize disk usage of the `compact` task: - -- [`granularitySpec`](#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup` -- [`dimensionsSpec`](#compaction-dimensions-spec) -- `metricsSpec` - -Compaction tasks exit without doing anything and issue a failure status code in either of the following cases: - -- If the interval you specify has no data segments loaded. -- If the interval you specify is empty. - -Note that the metadata between input segments and the resulting compacted segments may differ if the metadata among the input segments differs as well. If all input segments have the same metadata, however, the resulting output segment will have the same metadata as all input segments. - - -### Example compaction task - -The following JSON illustrates a compaction task to compact _all segments_ within the interval `2020-01-01/2021-01-01` and create new segments: - -```json -{ - "type": "compact", - "dataSource": "wikipedia", - "ioConfig": { - "type": "compact", - "inputSpec": { - "type": "interval", - "interval": "2020-01-01/2021-01-01" - } - }, - "granularitySpec": { - "segmentGranularity": "day", - "queryGranularity": "hour" - } -} -``` - -`granularitySpec` is an optional field. -If you don't specify `granularitySpec`, Druid retains the original segment and query granularities when compaction is complete. - -### Compaction I/O configuration - -The compaction `ioConfig` requires specifying `inputSpec` as follows: - -|Field|Description|Default|Required| -|-----|-----------|-------|--------| -|`type`|Task type. Set the value to `compact`.|none|Yes| -|`inputSpec`|Specification of the target [interval](#interval-inputspec) or [segments](#segments-inputspec).|none|Yes| -|`dropExisting`|If `true`, the task replaces all existing segments fully contained by either of the following:
- the `interval` in the `interval` type `inputSpec`.
- the umbrella interval of the `segments` in the `segment` type `inputSpec`.
If compaction fails, Druid does not change any of the existing segments.
**WARNING**: `dropExisting` in `ioConfig` is a beta feature. |false|No| -|`allowNonAlignedInterval`|If `true`, the task allows an explicit [`segmentGranularity`](#compaction-granularity-spec) that is not aligned with the provided [interval](#interval-inputspec) or [segments](#segments-inputspec). This parameter is only used if [`segmentGranularity`](#compaction-granularity-spec) is explicitly provided.

This parameter is provided for backwards compatibility. In most scenarios it should not be set, as it can lead to data being accidentally overshadowed. This parameter may be removed in a future release.|false|No| - -The compaction task has two kinds of `inputSpec`: - -#### Interval `inputSpec` - -|Field|Description|Required| -|-----|-----------|--------| -|`type`|Task type. Set the value to `interval`.|Yes| -|`interval`|Interval to compact.|Yes| - -#### Segments `inputSpec` - -|Field|Description|Required| -|-----|-----------|--------| -|`type`|Task type. Set the value to `segments`.|Yes| -|`segments`|A list of segment IDs.|Yes| - -### Compaction dimensions spec - -|Field|Description|Required| -|-----|-----------|--------| -|`dimensions`| A list of dimension names or objects. Cannot have the same column in both `dimensions` and `dimensionExclusions`. Defaults to `null`, which preserves the original dimensions.|No| -|`dimensionExclusions`| The names of dimensions to exclude from compaction. Only names are supported here, not objects. This list is only used if the dimensions list is null or empty; otherwise it is ignored. Defaults to `[]`.|No| - -### Compaction transform spec - -|Field|Description|Required| -|-----|-----------|--------| -|`filter`| The `filter` conditionally filters input rows during compaction. Only rows that pass the filter will be included in the compacted segments. Any of Druid's standard [query filters](../querying/filters.md) can be used. Defaults to 'null', which will not filter any row. |No| - -### Compaction granularity spec - -|Field|Description|Required| -|-----|-----------|--------| -|`segmentGranularity`|Time chunking period for the segment granularity. Defaults to 'null', which preserves the original segment granularity. Accepts all [Query granularity](../querying/granularities.md) values.|No| -|`queryGranularity`|The resolution of timestamp storage within each segment. Defaults to 'null', which preserves the original query granularity. Accepts all [Query granularity](../querying/granularities.md) values.|No| -|`rollup`|Enables compaction-time rollup. To preserve the original setting, keep the default value. To enable compaction-time rollup, set the value to `true`. Once the data is rolled up, you can no longer recover individual records.|No| - ## Learn more See the following topics for more information: - [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your case. +- [Manual compaction](./manual-compaction.md) for how to run a one-time compaction task - [Automatic compaction](automatic-compaction.md) for how to enable and configure automatic compaction. diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md new file mode 100644 index 000000000000..a2cd61b36b23 --- /dev/null +++ b/docs/data-management/manual-compaction.md @@ -0,0 +1,167 @@ +--- +id: manual-compaction +title: "Manual compaction" +--- + + + +In Apache Druid, compaction is a special type of ingestion task that reads data from a Druid datasource and writes it back into the same datasource. A common use case for this is to [optimally size segments](../operations/segment-optimization.md) after ingestion to improve query performance. + +You can perform manual compaction where you submit a one-time compaction task for a specific interval. Generally, you don't need to do this if you use [automatic compaction](./automatic-compaction.md), which is recommended for most workloads. + +## Setting up manual compaction + + Compaction tasks merge all segments for the defined interval according to the following syntax: + +```json +{ + "type": "compact", + "id": , + "dataSource": , + "ioConfig": , + "dimensionsSpec": , + "transformSpec": , + "metricsSpec": , + "tuningConfig": , + "granularitySpec": , + "context": +} +``` + +|Field|Description|Required| +|-----|-----------|--------| +|`type`|Task type. Set the value to `compact`.|Yes| +|`id`|Task ID|No| +|`dataSource`|Data source name to compact|Yes| +|`ioConfig`|I/O configuration for compaction task. See [Compaction I/O configuration](#compaction-io-configuration) for details.|Yes| +|`dimensionsSpec`|When set, the compaction task uses the specified `dimensionsSpec` rather than generating one from existing segments. See [Compaction dimensionsSpec](#compaction-dimensions-spec) for details.|No| +|`transformSpec`|When set, the compaction task uses the specified `transformSpec` rather than using `null`. See [Compaction transformSpec](#compaction-transform-spec) for details.|No| +|`metricsSpec`|When set, the compaction task uses the specified `metricsSpec` rather than generating one from existing segments.|No| +|`segmentGranularity`|Deprecated. Use `granularitySpec`.|No| +|`tuningConfig`|[Tuning configuration](../ingestion/native-batch.md#tuningconfig) for parallel indexing. `awaitSegmentAvailabilityTimeoutMillis` value is not supported for compaction tasks. Leave this parameter at the default value, 0.|No| +|`granularitySpec`|When set, the compaction task uses the specified `granularitySpec` rather than generating one from existing segments. See [Compaction `granularitySpec`](#compaction-granularity-spec) for details.|No| +|`context`|[Task context](../ingestion/tasks.md#context)|No| + +:::info + Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails. +::: + +To control the number of result segments per time chunk, you can set [`maxRowsPerSegment`](../ingestion/native-batch.md#partitionsspec) or [`numShards`](../ingestion/native-batch.md#tuningconfig). + +:::info + You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals. +::: + +A compaction task internally generates an `index` or `index_parallel` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [`druid` input source](../ingestion/input-sources.md), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default. + +Compaction tasks typically fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to maximize performance and minimize disk usage of the `compact` task: + +- [`granularitySpec`](#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup` +- [`dimensionsSpec`](#compaction-dimensions-spec) +- `metricsSpec` + +Compaction tasks exit without doing anything and issue a failure status code in either of the following cases: + +- If the interval you specify has no data segments loaded. +- If the interval you specify is empty. + +Note that the metadata between input segments and the resulting compacted segments may differ if the metadata among the input segments differs as well. If all input segments have the same metadata, however, the resulting output segment will have the same metadata as all input segments. + + +## Manual compaction task example + +The following JSON illustrates a compaction task to compact _all segments_ within the interval `2020-01-01/2021-01-01` and create new segments: + +```json +{ + "type": "compact", + "dataSource": "wikipedia", + "ioConfig": { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2020-01-01/2021-01-01" + } + }, + "granularitySpec": { + "segmentGranularity": "day", + "queryGranularity": "hour" + } +} +``` + +`granularitySpec` is an optional field. +If you don't specify `granularitySpec`, Druid retains the original segment and query granularities when compaction is complete. + +## Compaction I/O configuration + +The compaction `ioConfig` requires specifying `inputSpec` as follows: + +|Field|Description|Default|Required| +|-----|-----------|-------|--------| +|`type`|Task type. Set the value to `compact`.|none|Yes| +|`inputSpec`|Specification of the target [interval](#interval-inputspec) or [segments](#segments-inputspec).|none|Yes| +|`dropExisting`|If `true`, the task replaces all existing segments fully contained by either of the following:
- the `interval` in the `interval` type `inputSpec`.
- the umbrella interval of the `segments` in the `segment` type `inputSpec`.
If compaction fails, Druid does not change any of the existing segments.
**WARNING**: `dropExisting` in `ioConfig` is a beta feature. |false|No| +|`allowNonAlignedInterval`|If `true`, the task allows an explicit [`segmentGranularity`](#compaction-granularity-spec) that is not aligned with the provided [interval](#interval-inputspec) or [segments](#segments-inputspec). This parameter is only used if [`segmentGranularity`](#compaction-granularity-spec) is explicitly provided.

This parameter is provided for backwards compatibility. In most scenarios it should not be set, as it can lead to data being accidentally overshadowed. This parameter may be removed in a future release.|false|No| + +The compaction task has two kinds of `inputSpec`: + +### Interval `inputSpec` + +|Field|Description|Required| +|-----|-----------|--------| +|`type`|Task type. Set the value to `interval`.|Yes| +|`interval`|Interval to compact.|Yes| + +### Segments `inputSpec` + +|Field|Description|Required| +|-----|-----------|--------| +|`type`|Task type. Set the value to `segments`.|Yes| +|`segments`|A list of segment IDs.|Yes| + +## Compaction dimensions spec + +|Field|Description|Required| +|-----|-----------|--------| +|`dimensions`| A list of dimension names or objects. Cannot have the same column in both `dimensions` and `dimensionExclusions`. Defaults to `null`, which preserves the original dimensions.|No| +|`dimensionExclusions`| The names of dimensions to exclude from compaction. Only names are supported here, not objects. This list is only used if the dimensions list is null or empty; otherwise it is ignored. Defaults to `[]`.|No| + +## Compaction transform spec + +|Field|Description|Required| +|-----|-----------|--------| +|`filter`| The `filter` conditionally filters input rows during compaction. Only rows that pass the filter will be included in the compacted segments. Any of Druid's standard [query filters](../querying/filters.md) can be used. Defaults to 'null', which will not filter any row. |No| + +## Compaction granularity spec + +|Field|Description|Required| +|-----|-----------|--------| +|`segmentGranularity`|Time chunking period for the segment granularity. Defaults to 'null', which preserves the original segment granularity. Accepts all [Query granularity](../querying/granularities.md) values.|No| +|`queryGranularity`|The resolution of timestamp storage within each segment. Defaults to 'null', which preserves the original query granularity. Accepts all [Query granularity](../querying/granularities.md) values.|No| +|`rollup`|Enables compaction-time rollup. To preserve the original setting, keep the default value. To enable compaction-time rollup, set the value to `true`. Once the data is rolled up, you can no longer recover individual records.|No| + +## Learn more + +See the following topics for more information: +* [Compaction](compaction.md) for an overview of compaction and how to set up manual compaction in Druid. +* [Segment optimization](../operations/segment-optimization.md) for guidance on evaluating and optimizing Druid segment size. +* [Coordinator process](../design/coordinator.md#automatic-compaction) for details on how the Coordinator plans compaction tasks. + diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index bc02faf20061..017b4f38bec5 100644 --- a/docs/ingestion/ingestion-spec.md +++ b/docs/ingestion/ingestion-spec.md @@ -529,4 +529,4 @@ You can enable front coding with all types of ingestion. For information on defi ::: Beyond these properties, each ingestion method has its own specific tuning properties. See the documentation for each -[ingestion method](./index.md#ingestion-methods) for details. +[ingestion method](./index.md#ingestion-methods) for details. \ No newline at end of file diff --git a/website/sidebars.json b/website/sidebars.json index 1062b3dfee97..a38292bfafef 100644 --- a/website/sidebars.json +++ b/website/sidebars.json @@ -90,8 +90,18 @@ "data-management/update", "data-management/delete", "data-management/schema-changes", - "data-management/compaction", - "data-management/automatic-compaction" + { + "type": "category", + "label": "Compaction", + "link": { + "type": "doc", + "id": "data-management/compaction" + }, + "items": [ + "data-management/automatic-compaction", + "data-management/manual-compaction" + ] + } ], "Querying": [ { From f4a74710e6dc1eccfe376fcbfc1f488ea0fad8f5 Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Sun, 29 Oct 2023 12:10:49 +0100 Subject: [PATCH 10/15] Process pure ordering changes with windowing operators (#15241) - adds a new query build path: DruidQuery#toScanAndSortQuery which: - builds a ScanQuery without considering the current ordering - builds an operator to execute the sort - fixes a null string to "null" literal string conversion in the frame serializer code - fixes some DrillWindowQueryTest cases - fix NPE in NaiveSortOperator in case there was no input - enables back CoreRules.AGGREGATE_REMOVE - adds a processing level OffsetLimit class and uses that instead of just the limit in the rac parts - earlier window expressions on top of a subquery with an offset may have ignored the offset --- .../java/org/apache/druid/query/Druids.java | 5 + .../NaivePartitioningOperatorFactory.java | 20 + .../query/operator/NaiveSortOperator.java | 11 +- .../operator/NaiveSortOperatorFactory.java | 34 +- .../druid/query/operator/OffsetLimit.java | 143 +++ .../druid/query/operator/ScanOperator.java | 10 +- .../query/operator/ScanOperatorFactory.java | 42 +- .../query/operator/WindowOperatorQuery.java | 19 +- .../window/WindowOperatorFactory.java | 23 + .../window/ranking/WindowRankProcessor.java | 26 + .../ranking/WindowRankingProcessorBase.java | 24 + .../LazilyDecoratedRowsAndColumns.java | 35 +- .../DefaultColumnSelectorFactoryMaker.java | 3 + .../DefaultRowsAndColumnsDecorator.java | 19 +- .../semantic/RowsAndColumnsDecorator.java | 3 +- .../apache/druid/query/scan/ScanQuery.java | 7 + .../query/scan/ScanQueryQueryToolChest.java | 21 +- .../druid/segment/column/TypeStrategies.java | 3 + .../NaivePartitioningOperatorFactoryTest.java | 34 + .../NaiveSortOperatorFactoryTest.java | 34 + .../query/operator/NaiveSortOperatorTest.java | 98 ++ .../druid/query/operator/OffsetLimitTest.java | 109 ++ .../operator/ScanOperatorFactoryTest.java | 20 +- .../operator/WindowOperatorFactoryTest.java | 34 + .../operator/WindowOperatorQueryTest.java | 8 +- .../concrete/FrameRowsAndColumnsTest.java | 3 +- .../semantic/RowsAndColumnsDecoratorTest.java | 13 +- .../semantic/TestRowsAndColumnsDecorator.java | 11 +- ...ualColumnEvaluationRowsAndColumnsTest.java | 3 +- .../segment/column/TypeStrategiesTest.java | 10 +- .../calcite/planner/CalciteRulesManager.java | 14 +- .../sql/calcite/planner/OffsetLimit.java | 14 + .../sql/calcite/rel/DruidOuterQueryRel.java | 9 +- .../druid/sql/calcite/rel/DruidQuery.java | 91 +- .../druid/query/OperatorFactoryBuilders.java | 102 ++ .../query/WindowOperatorQueryBuilder.java | 91 ++ .../sql/calcite/CalciteArraysQueryTest.java | 81 +- .../sql/calcite/CalciteJoinQueryTest.java | 37 +- .../druid/sql/calcite/CalciteQueryTest.java | 327 ++++-- .../sql/calcite/CalciteWindowQueryTest.java | 23 +- .../sql/calcite/DrillWindowQueryTest.java | 58 +- .../druid/sql/calcite/NotYetSupported.java | 3 +- .../druid/sql/http/SqlResourceTest.java | 2 +- .../tests/window/offsetNotDiscarded.sqlTest | 31 + ...iaAggregationsMultipleOrderingDesc.sqlTest | 1004 ++++++++++++++++- 45 files changed, 2367 insertions(+), 345 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java create mode 100644 sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java create mode 100644 sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java create mode 100644 sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 24c1f3ddc4fe..f85e8daa52e9 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -868,6 +868,11 @@ public ScanQueryBuilder dataSource(String ds) dataSource = new TableDataSource(ds); return this; } + public ScanQueryBuilder dataSource(Query q) + { + dataSource = new QueryDataSource(q); + return this; + } public ScanQueryBuilder dataSource(DataSource ds) { diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java index 9bd937ab8443..c836007e77ef 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; public class NaivePartitioningOperatorFactory implements OperatorFactory { @@ -65,4 +66,23 @@ public String toString() "partitionColumns=" + partitionColumns + '}'; } + + @Override + public final int hashCode() + { + return Objects.hash(partitionColumns); + } + + @Override + public final boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || obj.getClass() != getClass()) { + return false; + } + NaivePartitioningOperatorFactory other = (NaivePartitioningOperatorFactory) obj; + return Objects.equals(partitionColumns, other.partitionColumns); + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java index e11da384210d..486d20482824 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java @@ -24,6 +24,7 @@ import java.io.Closeable; import java.util.ArrayList; +import java.util.List; /** * A naive sort operator is an operation that sorts a stream of data in-place. Generally speaking this means @@ -33,11 +34,11 @@ public class NaiveSortOperator implements Operator { private final Operator child; - private final ArrayList sortColumns; + private final List sortColumns; public NaiveSortOperator( Operator child, - ArrayList sortColumns + List sortColumns ) { this.child = child; @@ -57,7 +58,7 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) public Signal push(RowsAndColumns rac) { if (sorter == null) { - sorter = NaiveSortMaker.fromRAC(rac).make(sortColumns); + sorter = NaiveSortMaker.fromRAC(rac).make(new ArrayList<>(sortColumns)); } else { sorter.moreData(rac); } @@ -67,7 +68,9 @@ public Signal push(RowsAndColumns rac) @Override public void completed() { - receiver.push(sorter.complete()); + if (sorter != null) { + receiver.push(sorter.complete()); + } receiver.completed(); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java index 1ab80398b69b..623d0ed0fe5b 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java @@ -22,22 +22,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.ArrayList; +import java.util.List; +import java.util.Objects; public class NaiveSortOperatorFactory implements OperatorFactory { - private final ArrayList sortColumns; + private final List sortColumns; @JsonCreator public NaiveSortOperatorFactory( - @JsonProperty("columns") ArrayList sortColumns + @JsonProperty("columns") List sortColumns ) { this.sortColumns = sortColumns; } @JsonProperty("columns") - public ArrayList getSortColumns() + public List getSortColumns() { return sortColumns; } @@ -56,4 +57,29 @@ public boolean validateEquivalent(OperatorFactory other) } return false; } + + @Override + public int hashCode() + { + return Objects.hash(sortColumns); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + NaiveSortOperatorFactory other = (NaiveSortOperatorFactory) obj; + return Objects.equals(sortColumns, other.sortColumns); + } + + @Override + public String toString() + { + return "NaiveSortOperatorFactory{sortColumns=" + sortColumns + "}"; + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java b/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java new file mode 100644 index 000000000000..80fedf061159 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java @@ -0,0 +1,143 @@ +/* + * 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.operator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import java.util.Objects; + +public class OffsetLimit +{ + protected final long offset; + protected final long limit; + + public static final OffsetLimit NONE = new OffsetLimit(0, -1); + + @JsonCreator + public OffsetLimit( + @JsonProperty("offset") long offset, + @JsonProperty("limit") long limit) + { + Preconditions.checkArgument(offset >= 0, "offset >= 0"); + this.offset = offset; + this.limit = limit < 0 ? -1 : limit; + } + + @JsonProperty("offset") + public long getOffset() + { + return offset; + } + + @JsonProperty("limit") + public long getLimit() + { + return limit; + } + + public boolean isPresent() + { + return hasOffset() || hasLimit(); + } + + public boolean hasOffset() + { + return offset > 0; + } + + public boolean hasLimit() + { + return limit >= 0; + } + + public static OffsetLimit limit(int limit2) + { + return new OffsetLimit(0, limit2); + } + + public long getLimitOrMax() + { + if (limit < 0) { + return Long.MAX_VALUE; + } else { + return limit; + } + } + + @Override + public final boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof OffsetLimit)) { + return false; + } + OffsetLimit that = (OffsetLimit) o; + return limit == that.limit && offset == that.offset; + } + + @Override + public final int hashCode() + { + return Objects.hash(limit, offset); + } + + @Override + public String toString() + { + return "OffsetLimit{" + + "offset=" + offset + + ", limit=" + limit + + '}'; + } + + /** + * Returns the first row index to fetch. + * + * @param maxIndex maximal index accessible + */ + public long getFromIndex(long maxIndex) + { + if (maxIndex <= offset) { + return 0; + } + return offset; + } + + /** + * Returns the last row index to fetch (non-inclusive). + * + * @param maxIndex maximal index accessible + */ + public long getToIndex(long maxIndex) + { + if (maxIndex <= offset) { + return 0; + } + if (hasLimit()) { + long toIndex = limit + offset; + return Math.min(maxIndex, toIndex); + } else { + return maxIndex; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java index b823c30d22ed..fd72d1b6da9c 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java @@ -43,7 +43,7 @@ public class ScanOperator implements Operator private final Operator subOperator; private final Interval timeRange; private final Filter filter; - private final int limit; + private final OffsetLimit offsetLimit; private final List projectedColumns; private final VirtualColumns virtualColumns; private final List ordering; @@ -55,7 +55,7 @@ public ScanOperator( Interval timeRange, Filter filter, List ordering, - int limit + OffsetLimit offsetLimit ) { this.subOperator = subOperator; @@ -64,7 +64,7 @@ public ScanOperator( this.timeRange = timeRange; this.filter = filter; this.ordering = ordering; - this.limit = limit; + this.offsetLimit = offsetLimit == null ? OffsetLimit.NONE : offsetLimit; } @Nullable @@ -93,8 +93,8 @@ public Signal push(RowsAndColumns rac) decor.limitTimeRange(timeRange); } - if (limit > 0) { - decor.setLimit(limit); + if (offsetLimit.isPresent()) { + decor.setOffsetLimit(offsetLimit); } if (!(ordering == null || ordering.isEmpty())) { diff --git a/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java index a764984855e0..99453cf415af 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java @@ -31,7 +31,7 @@ public class ScanOperatorFactory implements OperatorFactory { private final Interval timeRange; private final DimFilter filter; - private final int limit; + private final OffsetLimit offsetLimit; private final List projectedColumns; private final VirtualColumns virtualColumns; private final List ordering; @@ -39,7 +39,7 @@ public class ScanOperatorFactory implements OperatorFactory public ScanOperatorFactory( @JsonProperty("timeRange") final Interval timeRange, @JsonProperty("filter") final DimFilter filter, - @JsonProperty("limit") final Integer limit, + @JsonProperty("offsetLimit") final OffsetLimit offsetLimit, @JsonProperty("projectedColumns") final List projectedColumns, @JsonProperty("virtualColumns") final VirtualColumns virtualColumns, @JsonProperty("ordering") final List ordering @@ -47,7 +47,7 @@ public ScanOperatorFactory( { this.timeRange = timeRange; this.filter = filter; - this.limit = limit == null ? -1 : limit; + this.offsetLimit = offsetLimit; this.projectedColumns = projectedColumns; this.virtualColumns = virtualColumns; this.ordering = ordering; @@ -66,9 +66,9 @@ public DimFilter getFilter() } @JsonProperty - public int getLimit() + public OffsetLimit getOffsetLimit() { - return limit; + return offsetLimit; } @JsonProperty @@ -99,7 +99,7 @@ public Operator wrap(Operator op) timeRange, filter == null ? null : filter.toFilter(), ordering, - limit + offsetLimit ); } @@ -119,18 +119,32 @@ public boolean equals(Object o) return false; } ScanOperatorFactory that = (ScanOperatorFactory) o; - return limit == that.limit && Objects.equals(timeRange, that.timeRange) && Objects.equals( - filter, - that.filter - ) && Objects.equals(projectedColumns, that.projectedColumns) && Objects.equals( - virtualColumns, - that.virtualColumns - ) && Objects.equals(ordering, that.ordering); + return Objects.equals(offsetLimit, that.offsetLimit) + && Objects.equals(timeRange, that.timeRange) + && Objects.equals(filter, that.filter) + && Objects.equals(projectedColumns, that.projectedColumns) + && Objects.equals(virtualColumns, that.virtualColumns) + && Objects.equals(ordering, that.ordering); } @Override public int hashCode() { - return Objects.hash(timeRange, filter, limit, projectedColumns, virtualColumns, ordering); + return Objects.hash(timeRange, filter, offsetLimit, projectedColumns, virtualColumns, ordering); } + + @Override + public String toString() + { + return "ScanOperatorFactory{" + + "timeRange=" + timeRange + + ", filter=" + filter + + ", offsetLimit=" + offsetLimit + + ", projectedColumns=" + projectedColumns + + ", virtualColumns=" + virtualColumns + + ", ordering=" + ordering + + "}"; + } + + } diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index 5ecba3f22420..50289b9851be 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -34,11 +34,13 @@ import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; + /** * A query that can compute window functions on top of a completely in-memory inline datasource or query results. *

@@ -122,14 +124,17 @@ public WindowOperatorQuery( ) ); } + if (ordering.isEmpty()) { + ordering = null; + } this.leafOperators.add( new ScanOperatorFactory( null, scan.getFilter(), - (int) scan.getScanRowsLimit(), + scan.getOffsetLimit(), scan.getColumns(), - scan.getVirtualColumns(), + scan.getVirtualColumns().isEmpty() ? null : scan.getVirtualColumns(), ordering ) ); @@ -242,16 +247,15 @@ public boolean equals(Object o) return false; } WindowOperatorQuery that = (WindowOperatorQuery) o; - return Objects.equals(rowSignature, that.rowSignature) && Objects.equals( - operators, - that.operators - ); + return Objects.equals(rowSignature, that.rowSignature) + && Objects.equals(operators, that.operators) + && Objects.equals(leafOperators, that.leafOperators); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), rowSignature, operators); + return Objects.hash(super.hashCode(), rowSignature, operators, leafOperators); } @Override @@ -263,6 +267,7 @@ public String toString() ", context=" + getContext() + ", rowSignature=" + rowSignature + ", operators=" + operators + + ", leafOperators=" + leafOperators + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java index ea44001d5078..88054bc1270b 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java @@ -26,6 +26,8 @@ import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowProcessorOperator; +import java.util.Objects; + public class WindowOperatorFactory implements OperatorFactory { private Processor processor; @@ -67,4 +69,25 @@ public String toString() "processor=" + processor + '}'; } + + @Override + public int hashCode() + { + return Objects.hash(processor); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || obj.getClass() != getClass()) { + return false; + } + WindowOperatorFactory other = (WindowOperatorFactory) obj; + return Objects.equals(processor, other.processor); + } + + } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java index 2d9b21863de8..b193398dfe15 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java @@ -28,6 +28,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Objects; /** * This Processor assumes that data has already been sorted for it. It does not re-sort the data and if it is given @@ -105,4 +106,29 @@ public String toString() ", asPercent=" + asPercent + '}'; } + + @Override + public int hashCode() + { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + Objects.hash(asPercent); + return result; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + WindowRankProcessor other = (WindowRankProcessor) obj; + return asPercent == other.asPercent; + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java index 4bff17174d67..fb5bedf9519f 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java @@ -28,6 +28,7 @@ import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.util.List; +import java.util.Objects; import java.util.function.Function; /** @@ -100,4 +101,27 @@ protected String internalToString() return "groupingCols=" + groupingCols + ", outputColumn='" + outputColumn + '\''; } + + @Override + public int hashCode() + { + return Objects.hash(groupingCols, outputColumn); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + WindowRankingProcessorBase other = (WindowRankingProcessorBase) obj; + return Objects.equals(groupingCols, other.groupingCols) && Objects.equals(outputColumn, other.outputColumn); + } + } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 79505dcdd419..3e938eb2d9d8 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -37,6 +37,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; @@ -73,7 +74,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns private Interval interval; private Filter filter; private VirtualColumns virtualColumns; - private int limit; + private OffsetLimit limit; private LinkedHashSet viewableColumns; private List ordering; @@ -82,7 +83,7 @@ public LazilyDecoratedRowsAndColumns( Interval interval, Filter filter, VirtualColumns virtualColumns, - int limit, + OffsetLimit limit, List ordering, LinkedHashSet viewableColumns ) @@ -175,7 +176,7 @@ private void maybeMaterialize() private boolean needsMaterialization() { - return interval != null || filter != null || limit != -1 || ordering != null || virtualColumns != null; + return interval != null || filter != null || limit.isPresent() || ordering != null || virtualColumns != null; } private Pair materialize() @@ -198,7 +199,7 @@ private void reset(RowsAndColumns rac) interval = null; filter = null; virtualColumns = null; - limit = -1; + limit = OffsetLimit.NONE; viewableColumns = null; ordering = null; } @@ -238,7 +239,8 @@ private Pair materializeStorageAdapter(StorageAdapter as) throw new ISE("accumulated[%s] non-null, why did we get multiple cursors?", accumulated); } - int theLimit = limit == -1 ? Integer.MAX_VALUE : limit; + long remainingRowsToSkip = limit.getOffset(); + long remainingRowsToFetch = limit.getLimitOrMax(); final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory(); final RowSignature.Builder sigBob = RowSignature.builder(); @@ -284,12 +286,12 @@ private Pair materializeStorageAdapter(StorageAdapter as) ); final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory); - while (!in.isDoneOrInterrupted()) { + for (; !in.isDoneOrInterrupted() && remainingRowsToSkip > 0; remainingRowsToSkip--) { + in.advance(); + } + for (; !in.isDoneOrInterrupted() && remainingRowsToFetch > 0; remainingRowsToFetch--) { frameWriter.addSelection(); in.advance(); - if (--theLimit <= 0) { - break; - } } return frameWriter; @@ -390,12 +392,8 @@ private Pair naiveMaterialize(RowsAndColumns rac) sigBob.add(column, racColumn.toAccessor().getType()); } - final int limitedNumRows; - if (limit == -1) { - limitedNumRows = Integer.MAX_VALUE; - } else { - limitedNumRows = limit; - } + long remainingRowsToSkip = limit.getOffset(); + long remainingRowsToFetch = limit.getLimitOrMax(); final FrameWriter frameWriter = FrameWriters.makeFrameWriterFactory( FrameType.COLUMNAR, @@ -405,11 +403,16 @@ private Pair naiveMaterialize(RowsAndColumns rac) ).newFrameWriter(selectorFactory); rowId.set(0); - for (; rowId.get() < numRows && frameWriter.getNumRows() < limitedNumRows; rowId.incrementAndGet()) { + for (; rowId.get() < numRows && remainingRowsToFetch > 0; rowId.incrementAndGet()) { final int theId = rowId.get(); if (rowsToSkip != null && rowsToSkip.get(theId)) { continue; } + if (remainingRowsToSkip > 0) { + remainingRowsToSkip--; + continue; + } + remainingRowsToFetch--; frameWriter.addSelection(); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java index 4611280ce89e..3c6d3cc08c9b 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java @@ -106,6 +106,9 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) protected String getValue() { final Object retVal = columnAccessor.getObject(cellIdSupplier.get()); + if (retVal == null) { + return null; + } if (retVal instanceof ByteBuffer) { return StringUtils.fromUtf8(((ByteBuffer) retVal).asReadOnlyBuffer()); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java index fd81491112de..3cfcfeec6142 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java @@ -21,6 +21,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.segment.VirtualColumn; @@ -39,14 +40,14 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator private Interval interval; private Filter filter; private VirtualColumns virtualColumns; - private int limit; + private OffsetLimit offsetLimit; private List ordering; public DefaultRowsAndColumnsDecorator( RowsAndColumns base ) { - this(base, null, null, null, -1, null); + this(base, null, null, null, OffsetLimit.NONE, null); } public DefaultRowsAndColumnsDecorator( @@ -54,7 +55,7 @@ public DefaultRowsAndColumnsDecorator( Interval interval, Filter filter, VirtualColumns virtualColumns, - int limit, + OffsetLimit limit, List ordering ) { @@ -62,7 +63,7 @@ public DefaultRowsAndColumnsDecorator( this.interval = interval; this.filter = filter; this.virtualColumns = virtualColumns; - this.limit = limit; + this.offsetLimit = limit; this.ordering = ordering; } @@ -111,13 +112,9 @@ public void addVirtualColumns(VirtualColumns virtualColumns) } @Override - public void setLimit(int numRows) + public void setOffsetLimit(OffsetLimit offsetLimit) { - if (this.limit == -1) { - this.limit = numRows; - } else { - this.limit = Math.min(limit, numRows); - } + this.offsetLimit = offsetLimit; } @Override @@ -134,7 +131,7 @@ public RowsAndColumns restrictColumns(List columns) interval, filter, virtualColumns, - limit, + offsetLimit, ordering, columns == null ? null : new LinkedHashSet<>(columns) ); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java index 77d8e2068dd5..b066fbe64f2d 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java @@ -21,6 +21,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.segment.VirtualColumns; import org.joda.time.Interval; @@ -61,7 +62,7 @@ static RowsAndColumnsDecorator fromRAC(RowsAndColumns rac) void addVirtualColumns(VirtualColumns virtualColumn); - void setLimit(int numRows); + void setOffsetLimit(OffsetLimit offsetLimit); void setOrdering(List ordering); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 57f32bc4398e..b897811d6071 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -37,6 +37,7 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.Queries; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -325,6 +326,11 @@ public long getScanRowsLimit() return scanRowsLimit; } + public OffsetLimit getOffsetLimit() + { + return new OffsetLimit(scanRowsOffset, scanRowsLimit); + } + /** * Returns whether this query is limited or not. Because {@link Long#MAX_VALUE} is used to signify unlimitedness, * this is equivalent to {@code getScanRowsLimit() != Long.Max_VALUE}. @@ -667,4 +673,5 @@ public boolean equals(Object obj) return obj instanceof Integer && (int) obj == DEFAULT_BATCH_SIZE; } } + } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 4d0885da00d8..8a21e7f9fb3a 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -42,8 +42,10 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.DataSource; import org.apache.druid.query.FrameSignaturePair; import org.apache.druid.query.GenericQueryMetricsFactory; +import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; @@ -57,6 +59,8 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.utils.CloseableUtils; +import javax.annotation.Nullable; + import java.io.Closeable; import java.util.ArrayList; import java.util.Iterator; @@ -196,8 +200,7 @@ public RowSignature resultArraySignature(final ScanQuery query) final ColumnCapabilities capabilities = virtualColumn.capabilities(c -> null, columnName); columnType = capabilities != null ? capabilities.toColumnType() : null; } else { - // Unknown type. In the future, it would be nice to have a way to fill these in. - columnType = null; + columnType = getDataSourceColumnType(query.getDataSource(), columnName); } builder.add(columnName, columnType); @@ -207,6 +210,20 @@ public RowSignature resultArraySignature(final ScanQuery query) } } + @Nullable + private ColumnType getDataSourceColumnType(DataSource dataSource, String columnName) + { + if (dataSource instanceof InlineDataSource) { + InlineDataSource inlineDataSource = (InlineDataSource) dataSource; + ColumnCapabilities caps = inlineDataSource.getRowSignature().getColumnCapabilities(columnName); + if (caps != null) { + return caps.toColumnType(); + } + } + // Unknown type. In the future, it would be nice to have a way to fill these in. + return null; + } + /** * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case * it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue} diff --git a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java index 20507c597b0f..3afcfdb7074c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java +++ b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java @@ -53,6 +53,9 @@ public class TypeStrategies @Nullable public static TypeStrategy getComplex(String typeName) { + if (typeName == null) { + return null; + } return COMPLEX_STRATEGIES.get(typeName); } diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java new file mode 100644 index 000000000000..123e6b4198e0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java @@ -0,0 +1,34 @@ +/* + * 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.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class NaivePartitioningOperatorFactoryTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NaivePartitioningOperatorFactory.class) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java new file mode 100644 index 000000000000..4c299c2e8c36 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java @@ -0,0 +1,34 @@ +/* + * 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.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class NaiveSortOperatorFactoryTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NaiveSortOperatorFactory.class) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java new file mode 100644 index 000000000000..3a54dd5f853f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.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.operator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.operator.Operator.Signal; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Test; + +public class NaiveSortOperatorTest +{ + @Test + public void testNoInputisHandledCorrectly() + { + NaiveSortOperator op = new NaiveSortOperator( + InlineScanOperator.make(), + ImmutableList.of(ColumnWithDirection.ascending("someColumn")) + ); + + new OperatorTestHelper() + .withPushFn(() -> (someRac) -> Signal.GO) + .runToCompletion(op); + } + + @Test + public void testSortAscending() + { + RowsAndColumns rac1 = racForColumn("c", new int[] {5, 3, 1}); + RowsAndColumns rac2 = racForColumn("c", new int[] {2, 6, 4}); + + NaiveSortOperator op = new NaiveSortOperator( + InlineScanOperator.make(rac1, rac2), + ImmutableList.of(ColumnWithDirection.ascending("c")) + ); + + new OperatorTestHelper() + .expectAndStopAfter( + new RowsAndColumnsHelper() + .expectColumn("c", new int[] {1, 2, 3, 4, 5, 6}) + ) + .runToCompletion(op); + } + + @Test + public void testSortDescending() + { + RowsAndColumns rac1 = racForColumn("c", new int[] {5, 3, 1}); + RowsAndColumns rac2 = racForColumn("c", new int[] {2, 6, 4}); + + NaiveSortOperator op = new NaiveSortOperator( + InlineScanOperator.make(rac1, rac2), + ImmutableList.of(ColumnWithDirection.descending("c")) + ); + + new OperatorTestHelper() + .expectAndStopAfter( + new RowsAndColumnsHelper() + .expectColumn("c", new int[] {6, 5, 4, 3, 2, 1}) + ) + .runToCompletion(op); + } + + private MapOfColumnsRowsAndColumns racForColumn(String k1, Object arr) + { + if (int.class.equals(arr.getClass().getComponentType())) { + return racForColumn(k1, new IntArrayColumn((int[]) arr)); + } + throw new IllegalArgumentException("Not yet supported"); + } + + private MapOfColumnsRowsAndColumns racForColumn(String k1, Column v1) + { + return MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of(k1, v1)); + } + +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java b/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java new file mode 100644 index 000000000000..f6fc6cd32c94 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java @@ -0,0 +1,109 @@ +/* + * 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.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class OffsetLimitTest +{ + @Test + public void testNone() + { + assertFalse(OffsetLimit.NONE.isPresent()); + assertFalse(OffsetLimit.NONE.hasOffset()); + assertFalse(OffsetLimit.NONE.hasLimit()); + } + + @Test + public void testOffset() + { + int offset = 3; + OffsetLimit ol = new OffsetLimit(offset, -1); + assertTrue(ol.hasOffset()); + assertFalse(ol.hasLimit()); + assertEquals(offset, ol.getOffset()); + assertEquals(-1, ol.getLimit()); + assertEquals(Long.MAX_VALUE, ol.getLimitOrMax()); + assertEquals(offset, ol.getFromIndex(Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, ol.getToIndex(Long.MAX_VALUE)); + assertEquals(0, ol.getFromIndex(1)); + assertEquals(0, ol.getFromIndex(offset)); + assertEquals(0, ol.getToIndex(offset)); + } + + @Test + public void testLimit() + { + OffsetLimit ol = new OffsetLimit(0, 4); + assertFalse(ol.hasOffset()); + assertTrue(ol.hasLimit()); + assertEquals(0, ol.getOffset()); + assertEquals(4, ol.getLimit()); + assertEquals(4, ol.getLimitOrMax()); + assertEquals(0, ol.getFromIndex(Long.MAX_VALUE)); + assertEquals(4, ol.getToIndex(Long.MAX_VALUE)); + assertEquals(0, ol.getFromIndex(2)); + assertEquals(2, ol.getToIndex(2)); + } + + @Test + public void testOffsetLimit() + { + int offset = 3; + int limit = 10; + OffsetLimit ol = new OffsetLimit(offset, limit); + assertTrue(ol.hasOffset()); + assertTrue(ol.hasLimit()); + assertEquals(offset, ol.getOffset()); + assertEquals(limit, ol.getLimit()); + assertEquals(limit, ol.getLimitOrMax()); + assertEquals(offset, ol.getFromIndex(Long.MAX_VALUE)); + assertEquals(offset + limit, ol.getToIndex(Long.MAX_VALUE)); + assertEquals(0, ol.getFromIndex(offset)); + assertEquals(0, ol.getToIndex(offset)); + assertEquals(offset, ol.getFromIndex(offset + 1)); + assertEquals(offset + 1, ol.getToIndex(offset + 1)); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidOffset() + { + new OffsetLimit(-1, -1); + } + + @Test + public void testNegativeLimitsAreNotDifferent() + { + OffsetLimit ol1 = new OffsetLimit(1, -1); + OffsetLimit ol2 = new OffsetLimit(1, -2); + assertEquals(ol1, ol2); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(OffsetLimit.class).verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java index 2b02d161d7a5..74bb30485659 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java @@ -61,7 +61,7 @@ public void testEquals() final Builder bob = new Builder(); bob.timeRange = Intervals.utc(0, 6); bob.filter = DimFilters.dimEquals("abc", "b"); - bob.limit = 48; + bob.offsetLimit = OffsetLimit.limit(48); bob.projectedColumns = Arrays.asList("a", "b"); bob.virtualColumns = VirtualColumns.EMPTY; bob.ordering = Collections.singletonList(ColumnWithDirection.ascending("a")); @@ -72,7 +72,7 @@ public void testEquals() Assert.assertNotEquals(factory, bob.copy().setTimeRange(null).build()); Assert.assertNotEquals(factory, bob.copy().setFilter(null).build()); - Assert.assertNotEquals(factory, bob.copy().setLimit(null).build()); + Assert.assertNotEquals(factory, bob.copy().setOffsetLimit(null).build()); Assert.assertNotEquals(factory, bob.copy().setProjectedColumns(null).build()); Assert.assertNotEquals(factory, bob.copy().setVirtualColumns(null).build()); Assert.assertNotEquals(factory, bob.copy().setOrdering(null).build()); @@ -132,7 +132,7 @@ public void testWrappedOperatorCarriesThroughValues() throws JsonProcessingExcep "interval[%s], filter[%s], limit[%s], ordering[%s], projection[%s], virtual[%s]", interval, filter, - limit, + OffsetLimit.limit(limit), ordering, projection, virtual @@ -141,7 +141,7 @@ public void testWrappedOperatorCarriesThroughValues() throws JsonProcessingExcep ScanOperatorFactory factory = new ScanOperatorFactory( interval, filter, - limit, + OffsetLimit.limit(limit), projection, virtual, ordering @@ -182,7 +182,7 @@ public Operator.Signal push(RowsAndColumns inRac) (TestRowsAndColumnsDecorator.DecoratedRowsAndColumns) inRac; Assert.assertEquals(msg, factory.getTimeRange(), rac.getTimeRange()); - Assert.assertEquals(msg, factory.getLimit(), rac.getLimit()); + Assert.assertEquals(msg, factory.getOffsetLimit(), rac.getOffsetLimit()); Assert.assertEquals(msg, factory.getVirtualColumns(), rac.getVirtualColumns()); validateList(msg, factory.getOrdering(), rac.getOrdering()); validateList(msg, factory.getProjectedColumns(), rac.getProjectedColumns()); @@ -228,7 +228,7 @@ private static class Builder { private Interval timeRange; private DimFilter filter; - private Integer limit; + private OffsetLimit offsetLimit; private List projectedColumns; private VirtualColumns virtualColumns; private List ordering; @@ -245,9 +245,9 @@ public Builder setFilter(DimFilter filter) return this; } - public Builder setLimit(Integer limit) + public Builder setOffsetLimit(OffsetLimit offsetLimit) { - this.limit = limit; + this.offsetLimit = offsetLimit; return this; } @@ -274,7 +274,7 @@ private Builder copy() Builder retVal = new Builder(); retVal.timeRange = timeRange; retVal.filter = filter; - retVal.limit = limit; + retVal.offsetLimit = offsetLimit; retVal.projectedColumns = projectedColumns; retVal.virtualColumns = virtualColumns; retVal.ordering = ordering; @@ -286,7 +286,7 @@ private ScanOperatorFactory build() return new ScanOperatorFactory( timeRange, filter, - limit, + offsetLimit, projectedColumns, virtualColumns, ordering diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java new file mode 100644 index 000000000000..7f1c6b00387a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java @@ -0,0 +1,34 @@ +/* + * 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.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class WindowOperatorFactoryTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NaivePartitioningOperatorFactory.class) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java index dcd969e57fa5..48b52cf781c1 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.QueryContext; @@ -131,8 +132,11 @@ public void withOperators() @Test public void testEquals() { - Assert.assertEquals(query, query); - Assert.assertEquals(query, query.withDataSource(query.getDataSource())); + EqualsVerifier.simple().forClass(WindowOperatorQuery.class) + .withNonnullFields("duration", "querySegmentSpec") + .usingGetClass() + .verify(); + Assert.assertNotEquals(query, query.toString()); } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java index 837e30185e2e..9bd529b195f8 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.rowsandcols.concrete; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase; @@ -38,7 +39,7 @@ public FrameRowsAndColumnsTest() private static FrameRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { - LazilyDecoratedRowsAndColumns rac = new LazilyDecoratedRowsAndColumns(input, null, null, null, Integer.MAX_VALUE, null, null); + LazilyDecoratedRowsAndColumns rac = new LazilyDecoratedRowsAndColumns(input, null, null, null, OffsetLimit.limit(Integer.MAX_VALUE), null, null); rac.numRows(); // materialize diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java index 3426dd009463..ad8967c1b5ba 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java @@ -29,6 +29,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; @@ -121,7 +122,7 @@ public void testDecoration() for (int k = 0; k <= limits.length; ++k) { int limit = (k == 0 ? -1 : limits[k - 1]); for (int l = 0; l <= orderings.length; ++l) { - validateDecorated(base, siggy, vals, interval, filter, limit, l == 0 ? null : orderings[l - 1]); + validateDecorated(base, siggy, vals, interval, filter, OffsetLimit.limit(limit), l == 0 ? null : orderings[l - 1]); } } } @@ -134,7 +135,7 @@ private void validateDecorated( Object[][] originalVals, Interval interval, Filter filter, - int limit, + OffsetLimit limit, List ordering ) { @@ -211,10 +212,10 @@ private void validateDecorated( vals.sort(comparator); } - if (limit != -1) { - decor.setLimit(limit); - - vals = vals.subList(0, Math.min(vals.size(), limit)); + if (limit.isPresent()) { + decor.setOffsetLimit(limit); + int size = vals.size(); + vals = vals.subList((int) limit.getFromIndex(size), (int) limit.getToIndex(vals.size())); } if (ordering != null) { diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java index ffddbb3f7430..79b4b51acf51 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java @@ -21,6 +21,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.VirtualColumns; @@ -35,7 +36,7 @@ public class TestRowsAndColumnsDecorator implements RowsAndColumnsDecorator private Interval timeRange; private Filter filter; private VirtualColumns virtualColumns; - private int limit = -1; + private OffsetLimit offsetLimit = OffsetLimit.NONE; private List ordering; private List projectedColumns; @@ -58,9 +59,9 @@ public void addVirtualColumns(VirtualColumns virtualColumns) } @Override - public void setLimit(int numRows) + public void setOffsetLimit(OffsetLimit offsetLimit) { - this.limit = numRows; + this.offsetLimit = offsetLimit; } @Override @@ -99,9 +100,9 @@ public VirtualColumns getVirtualColumns() return virtualColumns; } - public int getLimit() + public OffsetLimit getOffsetLimit() { - return limit; + return offsetLimit; } public List getOrdering() diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java index e53850bccc9a..c26508694d40 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; @@ -74,7 +75,7 @@ public void testMaterializeVirtualColumns() "val * 2", ColumnType.LONG, TestExprMacroTable.INSTANCE)), - Integer.MAX_VALUE, + OffsetLimit.NONE, null, null); diff --git a/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java b/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java index 66f4adcdd02d..19b49212ecf0 100644 --- a/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java @@ -35,6 +35,8 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import static org.junit.Assert.assertNull; + public class TypeStrategiesTest { ByteBuffer buffer = ByteBuffer.allocate(1 << 16); @@ -637,7 +639,7 @@ public int compareTo(NullableLongPair o) return Comparators.naturalNullsFirst().thenComparing(Longs::compare).compare(this.lhs, o.lhs); } } - + public static class NullableLongPairTypeStrategy implements TypeStrategy { @@ -692,4 +694,10 @@ public NullableLongPair fromBytes(byte[] value) return read(ByteBuffer.wrap(value)); } } + + @Test + public void getComplexTypeNull() + { + assertNull(TypeStrategies.getComplex(null)); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index b944f3cd5351..7095ea9275fc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -177,11 +177,7 @@ public class CalciteRulesManager private static final List ABSTRACT_RELATIONAL_RULES = ImmutableList.of( AbstractConverter.ExpandConversionRule.INSTANCE, - // Removing CoreRules.AGGREGATE_REMOVE rule here - // as after the Calcite upgrade, it would plan queries to a scan over a group by - // with ordering on a non-time column - // which is not allowed in Druid. We should add that rule back - // once Druid starts to support non-time ordering over scan queries + CoreRules.AGGREGATE_REMOVE, CoreRules.UNION_TO_DISTINCT, CoreRules.PROJECT_REMOVE, CoreRules.AGGREGATE_JOIN_TRANSPOSE, @@ -237,7 +233,13 @@ public List programs(final PlannerContext plannerContext) boolean isDebug = plannerContext.queryContext().isDebug(); return ImmutableList.of( - Programs.sequence(preProgram, Programs.ofRules(druidConventionRuleSet(plannerContext))), + Programs.sequence( + new LoggingProgram("Start", isDebug), + preProgram, + new LoggingProgram("After PreProgram", isDebug), + Programs.ofRules(druidConventionRuleSet(plannerContext)), + new LoggingProgram("After volcano planner program", isDebug) + ), Programs.sequence(preProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext))), Programs.sequence( // currently, adding logging program after every stage for easier debugging diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java index 8d4a375d558a..21d7e87b12dd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java @@ -76,6 +76,11 @@ public boolean hasLimit() return limit != null; } + public boolean isNone() + { + return !hasLimit() && !hasOffset(); + } + public long getLimit() { Preconditions.checkState(limit != null, "limit is not present"); @@ -162,4 +167,13 @@ public String toString() ", limit=" + limit + '}'; } + + public org.apache.druid.query.operator.OffsetLimit toOperatorOffsetLimit() + { + if (hasLimit()) { + return new org.apache.druid.query.operator.OffsetLimit(offset, limit); + } else { + return new org.apache.druid.query.operator.OffsetLimit(offset, -1); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java index c49fc6bd04e6..da828ce61ba6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java @@ -47,7 +47,14 @@ */ public class DruidOuterQueryRel extends DruidRel { - private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__subquery__"); + private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__subquery__") + { + @Override + public boolean isConcrete() + { + return false; + } + }; private static final QueryDataSource DUMMY_QUERY_DATA_SOURCE = new QueryDataSource( Druids.newScanQueryBuilder().dataSource("__subquery__").eternityInterval().build() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index b670d682222d..54ac7c364e2f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -67,6 +67,9 @@ import org.apache.druid.query.groupby.having.DimFilterHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.ColumnWithDirection.Direction; +import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.ScanOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; @@ -1014,11 +1017,16 @@ private Query computeQuery() return groupByQuery; } - final ScanQuery scanQuery = toScanQuery(); + final ScanQuery scanQuery = toScanQuery(true); if (scanQuery != null) { return scanQuery; } + final WindowOperatorQuery scanAndSortQuery = toScanAndSortQuery(); + if (scanAndSortQuery != null) { + return scanAndSortQuery; + } + throw new CannotBuildQueryException("Cannot convert query parts into an actual query"); } @@ -1439,6 +1447,11 @@ private WindowOperatorQuery toWindowQuery() if (windowing == null) { return null; } + + // This is not yet supported + if (dataSource.isConcrete()) { + return null; + } if (dataSource instanceof TableDataSource) { // We need a scan query to pull the results up for us before applying the window // Returning null here to ensure that the planner generates that alternative @@ -1473,13 +1486,83 @@ private WindowOperatorQuery toWindowQuery() ); } + /** + * Create an OperatorQuery which runs an order on top of a scan. + */ + @Nullable + private WindowOperatorQuery toScanAndSortQuery() + { + if (sorting == null + || sorting.getOrderBys().isEmpty() + || sorting.getProjection() != null) { + return null; + } + + ScanQuery scan = toScanQuery(false); + if (scan == null) { + return null; + } + + if (dataSource.isConcrete()) { + // Currently only non-time orderings of subqueries are allowed. + List orderByColumnNames = sorting.getOrderBys() + .stream().map(OrderByColumnSpec::getDimension) + .collect(Collectors.toList()); + plannerContext.setPlanningError( + "SQL query requires ordering a table by non-time column [%s], which is not supported.", + orderByColumnNames + ); + return null; + } + + QueryDataSource newDataSource = new QueryDataSource(scan); + List sortColumns = getColumnWithDirectionsFromOrderBys(sorting.getOrderBys()); + RowSignature signature = getOutputRowSignature(); + List operators = new ArrayList<>(); + + operators.add(new NaiveSortOperatorFactory(sortColumns)); + if (!sorting.getOffsetLimit().isNone()) { + operators.add( + new ScanOperatorFactory( + null, + null, + sorting.getOffsetLimit().toOperatorOffsetLimit(), + null, + null, + null + ) + ); + } + + return new WindowOperatorQuery( + newDataSource, + new LegacySegmentSpec(Intervals.ETERNITY), + plannerContext.queryContextMap(), + signature, + operators, + null + ); + } + + private ArrayList getColumnWithDirectionsFromOrderBys(List orderBys) + { + ArrayList ordering = new ArrayList<>(); + for (OrderByColumnSpec orderBySpec : orderBys) { + Direction direction = orderBySpec.getDirection() == OrderByColumnSpec.Direction.ASCENDING + ? ColumnWithDirection.Direction.ASC + : ColumnWithDirection.Direction.DESC; + ordering.add(new ColumnWithDirection(orderBySpec.getDimension(), direction)); + } + return ordering; + } + /** * Return this query as a Scan query, or null if this query is not compatible with Scan. - * + * @param considerSorting can be used to ignore the current sorting requirements {@link #toScanAndSortQuery()} uses it to produce the non-sorted part * @return query or null */ @Nullable - private ScanQuery toScanQuery() + private ScanQuery toScanQuery(final boolean considerSorting) { if (grouping != null || windowing != null) { // Scan cannot GROUP BY or do windows. @@ -1504,7 +1587,7 @@ private ScanQuery toScanQuery() long scanOffset = 0L; long scanLimit = 0L; - if (sorting != null) { + if (considerSorting && sorting != null) { scanOffset = sorting.getOffsetLimit().getOffset(); if (sorting.getOffsetLimit().hasLimit()) { diff --git a/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java b/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java new file mode 100644 index 000000000000..29aae495e8f9 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java @@ -0,0 +1,102 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.ColumnWithDirection.Direction; +import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; +import org.apache.druid.query.operator.NaiveSortOperatorFactory; +import org.apache.druid.query.operator.OffsetLimit; +import org.apache.druid.query.operator.OperatorFactory; +import org.apache.druid.query.operator.ScanOperatorFactory; +import org.apache.druid.query.operator.window.ComposingProcessor; +import org.apache.druid.query.operator.window.Processor; +import org.apache.druid.query.operator.window.WindowOperatorFactory; +import org.apache.druid.query.operator.window.ranking.WindowRankProcessor; + +import java.util.Arrays; +import java.util.List; + +public class OperatorFactoryBuilders +{ + + public static ScanOperatorFactoryBuilder scanOperatorFactoryBuilder() + { + return new ScanOperatorFactoryBuilder(); + } + + public static class ScanOperatorFactoryBuilder + { + private OffsetLimit offsetLimit; + private DimFilter filter; + private List projectedColumns; + + public OperatorFactory build() + { + return new ScanOperatorFactory(null, filter, offsetLimit, projectedColumns, null, null); + } + + public ScanOperatorFactoryBuilder setOffsetLimit(long offset, long limit) + { + offsetLimit = new OffsetLimit(offset, limit); + return this; + } + + public ScanOperatorFactoryBuilder setFilter(DimFilter filter) + { + this.filter = filter; + return this; + } + + public ScanOperatorFactoryBuilder setProjectedColumns(String... columns) + { + this.projectedColumns = Arrays.asList(columns); + return this; + } + } + + public static OperatorFactory naiveSortOperator(ColumnWithDirection... colWithDirs) + { + return new NaiveSortOperatorFactory(Arrays.asList(colWithDirs)); + } + + public static OperatorFactory naiveSortOperator(String column, Direction direction) + { + return naiveSortOperator(new ColumnWithDirection(column, direction)); + } + + public static OperatorFactory naivePartitionOperator(String... columns) + { + return new NaivePartitioningOperatorFactory(Arrays.asList(columns)); + } + + public static WindowOperatorFactory windowOperators(Processor... processors) + { + Preconditions.checkArgument(processors.length > 0, "You must specify at least one processor!"); + return new WindowOperatorFactory(processors.length == 1 ? processors[0] : new ComposingProcessor(processors)); + } + + public static Processor rankProcessor(String outputColumn, String... groupingColumns) + { + return new WindowRankProcessor(Arrays.asList(groupingColumns), outputColumn, false); + } +} diff --git a/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java b/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java new file mode 100644 index 000000000000..91ab769898c0 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java @@ -0,0 +1,91 @@ +/* + * 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; + +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.operator.OperatorFactory; +import org.apache.druid.query.operator.WindowOperatorQuery; +import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.column.RowSignature; + +import java.util.List; +import java.util.Map; + +public class WindowOperatorQueryBuilder +{ + private DataSource dataSource; + private QuerySegmentSpec intervals = new LegacySegmentSpec(Intervals.ETERNITY); + private Map context; + private RowSignature rowSignature; + private List operators; + private List leafOperators; + + public static WindowOperatorQueryBuilder builder() + { + return new WindowOperatorQueryBuilder(); + } + + public WindowOperatorQueryBuilder setDataSource(DataSource dataSource) + { + this.dataSource = dataSource; + return this; + } + + public WindowOperatorQueryBuilder setDataSource(String dataSource) + { + return setDataSource(new TableDataSource(dataSource)); + } + + public WindowOperatorQueryBuilder setDataSource(Query query) + { + return setDataSource(new QueryDataSource(query)); + } + + public WindowOperatorQueryBuilder setSignature(RowSignature rowSignature) + { + this.rowSignature = rowSignature; + return this; + } + + public Query build() + { + return new WindowOperatorQuery( + dataSource, + intervals, + context, + rowSignature, + operators, + leafOperators); + } + + public WindowOperatorQueryBuilder setOperators(OperatorFactory... operators) + { + this.operators = Lists.newArrayList(operators); + return this; + } + + public WindowOperatorQueryBuilder setLeafOperators(OperatorFactory... operators) + { + this.leafOperators = Lists.newArrayList(operators); + return this; + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index ec4dda0dc696..3a5da7d325ff 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -50,6 +50,7 @@ import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.SubstringDimExtractionFn; @@ -3171,55 +3172,45 @@ public void testArrayAggAsArrayFromJoin() public void testArrayAggGroupByArrayAggFromSubquery() { cannotVectorize(); - skipVectorize(); + testQuery( "SELECT dim2, arr, COUNT(*) FROM (SELECT dim2, ARRAY_AGG(DISTINCT dim1) as arr FROM foo WHERE dim1 is not null GROUP BY 1 LIMIT 5) GROUP BY 1,2", QUERY_CONTEXT_NO_STRINGIFY_ARRAY, ImmutableList.of( - GroupByQuery.builder() - .setDataSource(new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .dimension(new DefaultDimensionSpec( - "dim2", - "d0", - ColumnType.STRING - )) - .metric(new DimensionTopNMetricSpec( - null, - StringComparators.LEXICOGRAPHIC - )) - .filters(notNull("dim1")) - .threshold(5) - .aggregators(new ExpressionLambdaAggregatorFactory( - "a0", - ImmutableSet.of("dim1"), - "__acc", - "ARRAY[]", - "ARRAY[]", - true, - true, - false, - "array_set_add(\"__acc\", \"dim1\")", - "array_set_add_all(\"__acc\", \"a0\")", - null, - null, - new HumanReadableBytes(1024), - ExprMacroTable.nil() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .build() - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING), - new DefaultDimensionSpec("a0", "_d1", ColumnType.STRING_ARRAY) - ) - .setAggregatorSpecs(new CountAggregatorFactory("_a0")) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .dimension(new DefaultDimensionSpec( + "dim2", + "d0", + ColumnType.STRING + )) + .metric(new DimensionTopNMetricSpec( + null, + StringComparators.LEXICOGRAPHIC + )) + .filters(notNull("dim1")) + .threshold(5) + .aggregators(new ExpressionLambdaAggregatorFactory( + "a0", + ImmutableSet.of("dim1"), + "__acc", + "ARRAY[]", + "ARRAY[]", + true, + true, + false, + "array_set_add(\"__acc\", \"dim1\")", + "array_set_add_all(\"__acc\", \"a0\")", + null, + null, + new HumanReadableBytes(1024), + ExprMacroTable.nil() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .postAggregators(new ExpressionPostAggregator("s0", "1", null, ExprMacroTable.nil())) + .build() ), useDefault ? ImmutableList.of( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index ba55f7b4e4a7..0882f3c9cb12 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -5381,8 +5381,6 @@ public void testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults(Map queryContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); @@ -5399,32 +5397,21 @@ public void testPlanWithInFilterMoreThanInSubQueryThreshold() .dataSource( JoinDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(InlineDataSource.fromIterable( - ImmutableList.of( - new Object[]{4842L}, - new Object[]{4844L}, - new Object[]{4845L}, - new Object[]{14905L}, - new Object[]{4853L}, - new Object[]{29064L} - ), - RowSignature.builder() - .add("ROW_VALUE", ColumnType.LONG) - .build() - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimensions( - new DefaultDimensionSpec("ROW_VALUE", "d0", ColumnType.LONG) - ) - .setGranularity(Granularities.ALL) - .setLimitSpec(NoopLimitSpec.instance()) + InlineDataSource.fromIterable( + ImmutableList.of( + new Object[]{4842L}, + new Object[]{4844L}, + new Object[]{4845L}, + new Object[]{14905L}, + new Object[]{4853L}, + new Object[]{29064L} + ), + RowSignature.builder() + .add("ROW_VALUE", ColumnType.LONG) .build() ), "j0.", - "(\"l1\" == \"j0.d0\")", + "(\"l1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, ExprMacroTable.nil(), 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 948eea4c1ffc..b67db5dce419 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 @@ -39,11 +39,13 @@ import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.OperatorFactoryBuilders; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; +import org.apache.druid.query.WindowOperatorQueryBuilder; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; @@ -96,6 +98,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction; import org.apache.druid.query.lookup.RegisteredLookupExtractionFn; +import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQuery.ResultFormat; @@ -2725,7 +2728,7 @@ public void testGroupByWithSelectProjections() ); } - @NotYetSupported(Modes.CANNOT_CONVERT) + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @Test public void testGroupByWithSelectAndOrderByProjections() { @@ -2810,7 +2813,7 @@ public void testTopNWithSelectProjections() ); } - @NotYetSupported(Modes.CANNOT_CONVERT) + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @Test public void testTopNWithSelectAndOrderByProjections() { @@ -4692,7 +4695,7 @@ public void testSimpleAggregations() ); } - @NotYetSupported(Modes.CANNOT_CONVERT) + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @Test public void testGroupByWithSortOnPostAggregationDefault() { @@ -4724,7 +4727,7 @@ public void testGroupByWithSortOnPostAggregationDefault() ); } - @NotYetSupported(Modes.CANNOT_CONVERT) + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @Test public void testGroupByWithSortOnPostAggregationNoTopNConfig() { @@ -4768,7 +4771,7 @@ public void testGroupByWithSortOnPostAggregationNoTopNConfig() ); } - @NotYetSupported(Modes.CANNOT_CONVERT) + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @Test public void testGroupByWithSortOnPostAggregationNoTopNContext() { @@ -5370,7 +5373,7 @@ public void testUnplannableQueries() final Map queries = ImmutableMap.of( // SELECT query with order by non-__time. "SELECT dim1 FROM druid.foo ORDER BY dim1", - "SQL query requires order by non-time column [[dim1 ASC]], which is not supported.", + "SQL query requires ordering a table by non-time column [[dim1]], which is not supported.", // JOIN condition with not-equals (<>). "SELECT foo.dim1, foo.dim2, l.k, l.v\n" @@ -13949,31 +13952,14 @@ public void testSubqueryTypeMismatchWithLiterals() + "group by 1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE3) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setGranularity(Granularities.ALL) - .addDimension(new DefaultDimensionSpec( - "dim1", - "_d0", - ColumnType.STRING - )) - .addAggregator(new LongSumAggregatorFactory("a0", "l1")) - .build() - ) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setDimensions(new DefaultDimensionSpec("_d0", "d0", ColumnType.STRING)) - .setAggregatorSpecs(aggregators( - new FilteredAggregatorFactory( - new CountAggregatorFactory("_a0"), - useDefault ? - selector("a0", "0") : - equality("a0", 0, ColumnType.LONG) - ) - )) - .setGranularity(Granularities.ALL) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + .setDataSource(CalciteTests.DATASOURCE3) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .addDimension(new DefaultDimensionSpec("dim1", "_d0", ColumnType.STRING)) + .addAggregator(new LongSumAggregatorFactory("a0", "l1")) + .setPostAggregatorSpecs(ImmutableList.of( + expressionPostAgg("p0", "case_searched((\"a0\" == 0),1,0)"))) + .build() ), useDefault ? ImmutableList.of( @@ -14303,72 +14289,83 @@ public void testWindowingErrorWithoutFeatureFlag() assertThat(e, invalidSqlIs("The query contains window functions; To run these window functions, enable [WINDOW_FUNCTIONS] in query context. (line [1], column [13])")); } - + @Test public void testInGroupByLimitOutGroupByOrderBy() { skipVectorize(); cannotVectorize(); - testQuery( - "with t AS (SELECT m2, COUNT(m1) as trend_score\n" - + "FROM \"foo\"\n" - + "GROUP BY 1 \n" - + "LIMIT 10\n" - + ")\n" - + "select m2, (MAX(trend_score)) from t\n" - + "where m2 > 2\n" - + "GROUP BY 1 \n" - + "ORDER BY 2 DESC", - QUERY_CONTEXT_DEFAULT, - ImmutableList.of( - new GroupByQuery.Builder() + + testBuilder() + .sql( + "with t AS (SELECT m2, COUNT(m1) as trend_score\n" + + "FROM \"foo\"\n" + + "GROUP BY 1 \n" + + "LIMIT 10\n" + + ")\n" + + "select m2, (MAX(trend_score)) from t\n" + + "where m2 > 2\n" + + "GROUP BY 1 \n" + + "ORDER BY 2 DESC" + ) + .expectedQuery( + WindowOperatorQueryBuilder.builder() .setDataSource( new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .dimension(new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE)) .threshold(10) - .aggregators(aggregators( - useDefault - ? new CountAggregatorFactory("a0") - : new FilteredAggregatorFactory( - new CountAggregatorFactory("a0"), - notNull("m1") + .aggregators( + aggregators( + useDefault + ? new CountAggregatorFactory("a0") + : new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + notNull("m1") + ) ) - )) + ) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .context(OUTER_LIMIT_CONTEXT) .build() ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.DOUBLE) + .setSignature( + RowSignature.builder() + .add("d0", ColumnType.DOUBLE) + .add("a0", ColumnType.LONG) + .build() ) - .setDimFilter( - useDefault ? - bound("d0", "2", null, true, false, null, StringComparators.NUMERIC) : - new RangeFilter("d0", ColumnType.LONG, 2L, null, true, false, null) + .setOperators( + OperatorFactoryBuilders.naiveSortOperator("a0", ColumnWithDirection.Direction.DESC) ) - .setAggregatorSpecs(aggregators( - new LongMaxAggregatorFactory("_a0", "a0") - )) - .setLimitSpec( - DefaultLimitSpec - .builder() - .orderBy(new OrderByColumnSpec("_a0", Direction.DESCENDING, StringComparators.NUMERIC)) + .setLeafOperators( + OperatorFactoryBuilders.scanOperatorFactoryBuilder() + .setOffsetLimit(0, Long.MAX_VALUE) + .setFilter( + range( + "d0", + ColumnType.LONG, + 2L, + null, + true, + false + ) + ) + .setProjectedColumns("a0", "d0") .build() ) - .setContext(OUTER_LIMIT_CONTEXT) .build() - ), - ImmutableList.of( - new Object[]{3.0D, 1L}, - new Object[]{4.0D, 1L}, - new Object[]{5.0D, 1L}, - new Object[]{6.0D, 1L} ) - ); + .expectedResults( + ImmutableList.of( + new Object[] {3.0D, 1L}, + new Object[] {4.0D, 1L}, + new Object[] {5.0D, 1L}, + new Object[] {6.0D, 1L} + ) + ) + .run(); } @Test @@ -14376,8 +14373,7 @@ public void testInGroupByOrderByLimitOutGroupByOrderByLimit() { skipVectorize(); cannotVectorize(); - testQuery( - "with t AS (SELECT m2 as mo, COUNT(m1) as trend_score\n" + String sql = "with t AS (SELECT m2 as mo, COUNT(m1) as trend_score\n" + "FROM \"foo\"\n" + "GROUP BY 1\n" + "ORDER BY trend_score DESC\n" @@ -14385,56 +14381,167 @@ public void testInGroupByOrderByLimitOutGroupByOrderByLimit() + "select mo, (MAX(trend_score)) from t\n" + "where mo > 2\n" + "GROUP BY 1 \n" - + "ORDER BY 2 DESC LIMIT 2\n", - QUERY_CONTEXT_DEFAULT, - ImmutableList.of( - new GroupByQuery.Builder() + + "ORDER BY 2 DESC LIMIT 2 OFFSET 1\n"; + ImmutableList expectedResults = ImmutableList.of( + new Object[] {4.0D, 1L}, + new Object[] {5.0D, 1L} + ); + + testBuilder() + .sql(sql) + .expectedQuery( + WindowOperatorQueryBuilder.builder() .setDataSource( new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .dimension(new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE)) .threshold(10) - .aggregators(aggregators( - useDefault - ? new CountAggregatorFactory("a0") - : new FilteredAggregatorFactory( - new CountAggregatorFactory("a0"), - notNull("m1") + .aggregators( + aggregators( + useDefault + ? new CountAggregatorFactory("a0") + : new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + notNull("m1") + ) ) - )) + ) .metric(new NumericTopNMetricSpec("a0")) .context(OUTER_LIMIT_CONTEXT) .build() ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.DOUBLE) + .setSignature( + RowSignature.builder() + .add("d0", ColumnType.DOUBLE) + .add("a0", ColumnType.LONG) + .build() ) - .setDimFilter( - useDefault ? - bound("d0", "2", null, true, false, null, StringComparators.NUMERIC) : - new RangeFilter("d0", ColumnType.LONG, 2L, null, true, false, null) + .setOperators( + OperatorFactoryBuilders.naiveSortOperator("a0", ColumnWithDirection.Direction.DESC), + OperatorFactoryBuilders.scanOperatorFactoryBuilder() + .setOffsetLimit(1, 2) + .build() ) - .setAggregatorSpecs(aggregators( - new LongMaxAggregatorFactory("_a0", "a0") - )) - .setLimitSpec( - DefaultLimitSpec - .builder() - .orderBy(new OrderByColumnSpec("_a0", Direction.DESCENDING, StringComparators.NUMERIC)) - .limit(2) + .setLeafOperators( + OperatorFactoryBuilders.scanOperatorFactoryBuilder() + .setOffsetLimit(0, Long.MAX_VALUE) + .setFilter( + range( + "d0", + ColumnType.LONG, + 2L, + null, + true, + false + ) + ) + .setProjectedColumns("a0", "d0") .build() ) - .setContext(OUTER_LIMIT_CONTEXT) .build() - ), - ImmutableList.of( - new Object[]{3.0D, 1L}, - new Object[]{4.0D, 1L} ) - ); + .expectedResults(expectedResults) + .run(); } + @NotYetSupported(Modes.CANNOT_TRANSLATE) + @Test + public void testWindowingWithScanAndSort() + { + skipVectorize(); + cannotVectorize(); + msqIncompatible(); + String sql = "with t AS (\n" + + "SELECT \n" + + " RANK() OVER (PARTITION BY m2 ORDER BY m2 ASC) \n" + + " AS ranking,\n" + + " COUNT(m1) as trend_score\n" + + "FROM foo\n" + + "GROUP BY m2,m1 LIMIT 10\n" + + ")\n" + + "select ranking, trend_score from t ORDER BY trend_score"; + ImmutableList expectedResults = ImmutableList.of( + new Object[] {1L, 1L}, + new Object[] {1L, 1L}, + new Object[] {1L, 1L}, + new Object[] {1L, 1L}, + new Object[] {1L, 1L}, + new Object[] {1L, 1L} + ); + + testBuilder() + .sql(sql) + .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) + .expectedQuery( + WindowOperatorQueryBuilder.builder() + .setDataSource( + Druids.newScanQueryBuilder() + .dataSource( + new WindowOperatorQueryBuilder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE), + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) + ) + ) + .setAggregatorSpecs( + aggregators( + useDefault + ? new CountAggregatorFactory("a0") + : new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + notNull("m1") + ) + ) + ) + .build() + ) + .setOperators( + OperatorFactoryBuilders.naivePartitionOperator("d0"), + OperatorFactoryBuilders.windowOperators( + OperatorFactoryBuilders.rankProcessor("w0", "d0") + ) + ) + .setSignature( + RowSignature.builder() + .add("w0", ColumnType.LONG) + .add("a0", ColumnType.LONG) + .build() + ) + .build() + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a0", "w0") + .context(QUERY_CONTEXT_DEFAULT) + .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .limit(10) + .build() + ) + .setSignature( + RowSignature.builder() + .add("w0", ColumnType.LONG) + .add("a0", ColumnType.LONG) + .build() + ) + .setOperators( + OperatorFactoryBuilders.naiveSortOperator("a0", ColumnWithDirection.Direction.ASC) + ) + .setLeafOperators( + OperatorFactoryBuilders.scanOperatorFactoryBuilder() + .setOffsetLimit(0, Long.MAX_VALUE) + .setProjectedColumns("a0", "w0") + .build() + ) + .build() + ) + .expectedResults(expectedResults) + .run(); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index b0172fcd0c8f..765dab45b528 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -128,18 +128,21 @@ public void verifyResults(QueryResults results) throws Exception Assert.assertEquals(1, results.recordedQueries.size()); maybeDumpActualResults(results.results); - final WindowOperatorQuery query = getWindowOperatorQuery(results.recordedQueries); - for (int i = 0; i < input.expectedOperators.size(); ++i) { - final OperatorFactory expectedOperator = input.expectedOperators.get(i); - final OperatorFactory actualOperator = query.getOperators().get(i); - if (!expectedOperator.validateEquivalent(actualOperator)) { - assertEquals("Operator Mismatch, index[" + i + "]", - queryJackson.writeValueAsString(expectedOperator), - queryJackson.writeValueAsString(actualOperator)); - fail("validateEquivalent failed; but textual comparision of operators didn't reported the mismatch!"); + if (input.expectedOperators != null) { + final WindowOperatorQuery query = getWindowOperatorQuery(results.recordedQueries); + for (int i = 0; i < input.expectedOperators.size(); ++i) { + final OperatorFactory expectedOperator = input.expectedOperators.get(i); + final OperatorFactory actualOperator = query.getOperators().get(i); + if (!expectedOperator.validateEquivalent(actualOperator)) { + assertEquals("Operator Mismatch, index[" + i + "]", + queryJackson.writeValueAsString(expectedOperator), + queryJackson.writeValueAsString(actualOperator)); + fail("validateEquivalent failed; but textual comparision of operators didn't reported the mismatch!"); + } } } - final RowSignature outputSignature = query.getRowSignature(); + + final RowSignature outputSignature = results.signature; ColumnType[] types = new ColumnType[outputSignature.size()]; for (int i = 0; i < outputSignature.size(); ++i) { types[i] = outputSignature.getColumnType(i).get(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 58138ffd1ee6..55014a2f414c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -383,7 +383,7 @@ public void verify(String sql, QueryResults queryResults) private boolean isOrdered(QueryResults queryResults) { - SqlNode sqlNode = ((PlannerCaptureHook) queryResults.capture).getSqlNode(); + SqlNode sqlNode = queryResults.capture.getSqlNode(); return SqlToRelConverter.isOrdered(sqlNode); } } @@ -4364,6 +4364,7 @@ public void test_aggregates_winFnQry_9() windowQueryTest(); } + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @DrillTest("nestedAggs/multiWin_5") @Test public void test_nestedAggs_multiWin_5() @@ -4477,7 +4478,6 @@ public void test_nestedAggs_multiWin_7() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("aggregates/aggOWnFn_3") @Test public void test_aggregates_aggOWnFn_3() @@ -4485,7 +4485,6 @@ public void test_aggregates_aggOWnFn_3() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("aggregates/aggOWnFn_4") @Test public void test_aggregates_aggOWnFn_4() @@ -4493,7 +4492,6 @@ public void test_aggregates_aggOWnFn_4() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("first_val/firstValFn_29") @Test public void test_first_val_firstValFn_29() @@ -4501,7 +4499,6 @@ public void test_first_val_firstValFn_29() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("first_val/firstValFn_32") @Test public void test_first_val_firstValFn_32() @@ -4509,7 +4506,7 @@ public void test_first_val_firstValFn_32() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("first_val/firstValFn_33") @Test public void test_first_val_firstValFn_33() @@ -4525,7 +4522,6 @@ public void test_frameclause_defaultFrame_RBUPACR_int7() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("lag_func/lag_Fn_9") @Test public void test_lag_func_lag_Fn_9() @@ -4533,7 +4529,6 @@ public void test_lag_func_lag_Fn_9() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("last_val/lastValFn_29") @Test public void test_last_val_lastValFn_29() @@ -4541,7 +4536,7 @@ public void test_last_val_lastValFn_29() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("last_val/lastValFn_34") @Test public void test_last_val_lastValFn_34() @@ -4549,7 +4544,7 @@ public void test_last_val_lastValFn_34() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("last_val/lastValFn_35") @Test public void test_last_val_lastValFn_35() @@ -4557,7 +4552,7 @@ public void test_last_val_lastValFn_35() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("last_val/lastValFn_38") @Test public void test_last_val_lastValFn_38() @@ -4565,7 +4560,7 @@ public void test_last_val_lastValFn_38() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("last_val/lastValFn_39") @Test public void test_last_val_lastValFn_39() @@ -4581,7 +4576,6 @@ public void test_nestedAggs_emtyOvrCls_7() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("ntile_func/ntileFn_33") @Test public void test_ntile_func_ntileFn_33() @@ -4589,7 +4583,6 @@ public void test_ntile_func_ntileFn_33() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("ntile_func/ntileFn_34") @Test public void test_ntile_func_ntileFn_34() @@ -4597,7 +4590,7 @@ public void test_ntile_func_ntileFn_34() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_COUNT_MISMATCH) @DrillTest("ntile_func/ntileFn_47") @Test public void test_ntile_func_ntileFn_47() @@ -4605,7 +4598,7 @@ public void test_ntile_func_ntileFn_47() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_COUNT_MISMATCH) @DrillTest("ntile_func/ntileFn_48") @Test public void test_ntile_func_ntileFn_48() @@ -4613,7 +4606,7 @@ public void test_ntile_func_ntileFn_48() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_COUNT_MISMATCH) @DrillTest("ntile_func/ntileFn_49") @Test public void test_ntile_func_ntileFn_49() @@ -4621,7 +4614,7 @@ public void test_ntile_func_ntileFn_49() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_COUNT_MISMATCH) @DrillTest("ntile_func/ntileFn_50") @Test public void test_ntile_func_ntileFn_50() @@ -4629,7 +4622,7 @@ public void test_ntile_func_ntileFn_50() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_COUNT_MISMATCH) @DrillTest("ntile_func/ntileFn_51") @Test public void test_ntile_func_ntileFn_51() @@ -4637,7 +4630,7 @@ public void test_ntile_func_ntileFn_51() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_52") @Test public void test_ntile_func_ntileFn_52() @@ -4645,7 +4638,7 @@ public void test_ntile_func_ntileFn_52() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_53") @Test public void test_ntile_func_ntileFn_53() @@ -4653,7 +4646,7 @@ public void test_ntile_func_ntileFn_53() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_54") @Test public void test_ntile_func_ntileFn_54() @@ -4661,7 +4654,7 @@ public void test_ntile_func_ntileFn_54() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_55") @Test public void test_ntile_func_ntileFn_55() @@ -4669,7 +4662,7 @@ public void test_ntile_func_ntileFn_55() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_56") @Test public void test_ntile_func_ntileFn_56() @@ -4677,7 +4670,7 @@ public void test_ntile_func_ntileFn_56() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_57") @Test public void test_ntile_func_ntileFn_57() @@ -4685,7 +4678,7 @@ public void test_ntile_func_ntileFn_57() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("ntile_func/ntileFn_58") @Test public void test_ntile_func_ntileFn_58() @@ -6697,7 +6690,6 @@ public void test_frameclause_defaultFrame_RBUPACR_bgint_7() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/defaultFrame/RBUPACR_chr_3") @Test public void test_frameclause_defaultFrame_RBUPACR_chr_3() @@ -6822,7 +6814,6 @@ public void test_frameclause_defaultFrame_RBUPACR_int9() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/defaultFrame/RBUPACR_vchr_3") @Test public void test_frameclause_defaultFrame_RBUPACR_vchr_3() @@ -6846,7 +6837,6 @@ public void test_frameclause_multipl_wnwds_avg_mulwds() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/multipl_wnwds/count_mulwds") @Test public void test_frameclause_multipl_wnwds_count_mulwds() @@ -6910,7 +6900,6 @@ public void test_frameclause_RBCRACR_RBCRACR_bgint_7() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBCRACR/RBCRACR_char_3") @Test public void test_frameclause_RBCRACR_RBCRACR_char_3() @@ -7012,7 +7001,6 @@ public void test_frameclause_RBCRACR_RBCRACR_int_7() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBCRACR/RBCRACR_vchar_3") @Test public void test_frameclause_RBCRACR_RBCRACR_vchar_3() @@ -7083,7 +7071,6 @@ public void test_frameclause_RBUPACR_RBUPACR_bgint_7() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBUPACR/RBUPACR_chr_3") @Test public void test_frameclause_RBUPACR_RBUPACR_chr_3() @@ -7161,7 +7148,6 @@ public void test_frameclause_RBUPACR_RBUPACR_int14() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBUPACR/RBUPACR_vchr_3") @Test public void test_frameclause_RBUPACR_RBUPACR_vchr_3() @@ -7192,7 +7178,6 @@ public void test_frameclause_RBUPAUF_RBUPAUF_bgint_6() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBUPAUF/RBUPAUF_char_3") @Test public void test_frameclause_RBUPAUF_RBUPAUF_char_3() @@ -7249,7 +7234,6 @@ public void test_frameclause_RBUPAUF_RBUPAUF_int_13() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/RBUPAUF/RBUPAUF_vchar_3") @Test public void test_frameclause_RBUPAUF_RBUPAUF_vchar_3() @@ -7257,7 +7241,6 @@ public void test_frameclause_RBUPAUF_RBUPAUF_vchar_3() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_53") @Test public void test_frameclause_subQueries_frmInSubQry_53() @@ -7265,7 +7248,6 @@ public void test_frameclause_subQueries_frmInSubQry_53() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_54") @Test public void test_frameclause_subQueries_frmInSubQry_54() @@ -7273,7 +7255,6 @@ public void test_frameclause_subQueries_frmInSubQry_54() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_55") @Test public void test_frameclause_subQueries_frmInSubQry_55() @@ -7623,7 +7604,6 @@ public void test_nestedAggs_cte_win_05() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("nestedAggs/emtyOvrCls_13") @Test public void test_nestedAggs_emtyOvrCls_13() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 86e5c41d6de8..7f4e6a06993c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -89,7 +89,8 @@ enum Modes // at least c7 is represented oddly in the parquet file T_ALLTYPES_ISSUES(AssertionError.class, "(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"), RESULT_MISMATCH(AssertionError.class, "assertResultsEquals"), - UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"); + UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"), + CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference"); public Class throwableClass; public String regex; diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 6e38c26e4f36..4d75cac5ae05 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -1391,7 +1391,7 @@ public void testCannotConvert() throws Exception DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "Query could not be planned. A possible reason is " - + "[SQL query requires order by non-time column [[dim1 ASC]], which is not supported.]" + + "[SQL query requires ordering a table by non-time column [[dim1]], which is not supported.]" ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); diff --git a/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest b/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest new file mode 100644 index 000000000000..060366d934b6 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest @@ -0,0 +1,31 @@ +type: "operatorValidation" + +sql: | + SELECT + RANK() OVER (PARTITION BY m1 ORDER BY m2 ASC) AS ranking, + m1,m2,dim1,dim2 + FROM foo + + +expectedOperators: + - type: "naiveSort" + columns: + - column: "m1" + direction: "ASC" + - column: "m2" + direction: "ASC" + - { type: "naivePartition", partitionColumns: [ m1 ] } + - type: "window" + processor: + type: "rank" + group: [ m2 ] + outputColumn: w0 + asPercent: false + +expectedResults: + - [1,1.0,1.0,"","a"] + - [1,2.0,2.0,"10.1",null] + - [1,3.0,3.0,"2",""] + - [1,4.0,4.0,"1","a"] + - [1,5.0,5.0,"def","abc"] + - [1,6.0,6.0,"abc",null] diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest index affed86e6b8d..0180f6153130 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest @@ -1,4 +1,4 @@ -type: "failingTest" +type: "operatorValidation" sql: | SELECT @@ -11,15 +11,993 @@ sql: | GROUP BY 1, 2 ORDER BY 1 DESC, 2 DESC -expectedOperators: - - { type: "naivePartition", partitionColumns: [ "d0" ] } - - type: "window" - processor: - type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: false, lowOffset: 3, uppUnbounded: false, uppOffset: 2 } - aggregations: - - { type: "longSum", name: "w0", fieldName: "a0" } - - { type: "naiveSort", columns: [ { column: "d1", direction: "DESC" }, { column: "a0", direction: "DESC"} ]} - - { type: "naivePartition", partitionColumns: [ "d1" ] } - - type: "window" - processor: { type: "rowNumber", outputColumn: "w1"} \ No newline at end of file +# expectedOperators are not validated as the query right now have 2 windowOperatorQuery objects + +expectedResults: + - ["ZW",1442048400000,254,254,16] + - ["ZW",1442044800000,0,254,33] + - ["ZM",1442041200000,133,133,14] + - ["ZA",1442091600000,1,51,32] + - ["ZA",1442070000000,0,130,44] + - ["ZA",1442059200000,50,127,25] + - ["ZA",1442048400000,79,127,22] + - ["ZA",1442034000000,-3,126,30] + - ["VN",1442084400000,-10,1280,46] + - ["VN",1442070000000,479,1426,11] + - ["VN",1442066400000,811,1434,11] + - ["VN",1442062800000,146,1471,17] + - ["VN",1442059200000,8,1571,32] + - ["VN",1442055600000,37,1077,26] + - ["VN",1442052000000,90,266,23] + - ["VN",1442048400000,-15,109,41] + - ["VN",1442041200000,0,72,37] + - ["VN",1442037600000,-11,98,33] + - ["VN",1442034000000,-29,-1,33] + - ["VN",1442026800000,63,14,22] + - ["VN",1442023200000,-9,14,30] + - ["VG",1442062800000,-238,-238,42] + - ["VE",1442098800000,9,104,22] + - ["VE",1442095200000,35,516,21] + - ["VE",1442084400000,60,936,27] + - ["VE",1442077200000,412,954,17] + - ["VE",1442070000000,420,943,13] + - ["VE",1442066400000,18,959,31] + - ["VE",1442034000000,-2,882,29] + - ["VE",1442030400000,51,585,17] + - ["VE",1442026800000,-17,165,32] + - ["VE",1442023200000,115,147,16] + - ["UZ",1442044800000,1369,1369,6] + - ["UY",1442077200000,23,265,30] + - ["UY",1442073600000,-42,266,41] + - ["UY",1442070000000,284,342,17] + - ["UY",1442037600000,1,859,29] + - ["UY",1442026800000,76,913,21] + - ["UY",1442023200000,517,955,10] + - ["UY",1442019600000,77,671,17] + - ["US",1442098800000,3575,6493,3] + - ["US",1442095200000,416,8184,11] + - ["US",1442091600000,2502,10707,6] + - ["US",1442088000000,1691,14708,8] + - ["US",1442084400000,2523,13301,5] + - ["US",1442080800000,4001,13985,4] + - ["US",1442077200000,2168,14988,9] + - ["US",1442073600000,1100,14069,13] + - ["US",1442070000000,3505,11593,4] + - ["US",1442066400000,772,7603,12] + - ["US",1442062800000,47,5591,22] + - ["US",1442059200000,11,4489,30] + - ["US",1442055600000,156,518,20] + - ["US",1442052000000,-2,-115,41] + - ["US",1442048400000,-466,1837,46] + - ["US",1442044800000,139,5501,19] + - ["US",1442041200000,1999,8993,2] + - ["US",1442037600000,3675,11018,3] + - ["US",1442034000000,3648,12996,2] + - ["US",1442030400000,2023,15701,4] + - ["US",1442026800000,1512,14745,5] + - ["US",1442023200000,2844,11070,2] + - ["US",1442019600000,1043,7422,5] + - ["US",1442016000000,0,5399,10] + - ["UG",1442070000000,1,1,42] + - ["UA",1442098800000,38,-380,17] + - ["UA",1442095200000,-30,-401,40] + - ["UA",1442091600000,-388,-396,41] + - ["UA",1442088000000,-21,-397,33] + - ["UA",1442084400000,5,-616,37] + - ["UA",1442080800000,-1,3655,39] + - ["UA",1442077200000,-181,5776,46] + - ["UA",1442073600000,4241,6093,5] + - ["UA",1442070000000,1733,6861,7] + - ["UA",1442066400000,296,6860,18] + - ["UA",1442062800000,773,21243,9] + - ["UA",1442059200000,-2,17412,38] + - ["UA",1442055600000,14202,15681,2] + - ["UA",1442052000000,410,15665,13] + - ["UA",1442048400000,2,14966,36] + - ["UA",1442044800000,280,14967,15] + - ["UA",1442041200000,74,4233,19] + - ["UA",1442037600000,-1,3823,31] + - ["UA",1442034000000,3468,3821,3] + - ["TW",1442098800000,-60,-137,39] + - ["TW",1442095200000,-77,-113,41] + - ["TW",1442084400000,0,389,39] + - ["TW",1442080800000,24,1161,24] + - ["TW",1442077200000,502,1706,16] + - ["TW",1442073600000,772,2407,15] + - ["TW",1442070000000,485,2135,10] + - ["TW",1442066400000,624,1954,14] + - ["TW",1442062800000,-272,1500,43] + - ["TW",1442059200000,-157,752,44] + - ["TW",1442055600000,48,342,24] + - ["TW",1442052000000,24,-258,32] + - ["TW",1442048400000,75,380,23] + - ["TW",1442044800000,24,803,27] + - ["TW",1442041200000,366,898,9] + - ["TW",1442037600000,266,874,9] + - ["TW",1442034000000,143,1479,18] + - ["TW",1442030400000,0,1552,26] + - ["TW",1442026800000,680,1186,9] + - ["TW",1442023200000,97,1012,19] + - ["TW",1442019600000,0,869,32] + - ["TW",1442016000000,92,869,5] + - ["TT",1442088000000,9,9,30] + - ["TR",1442095200000,-29,5408,39] + - ["TR",1442091600000,3048,5578,4] + - ["TR",1442088000000,2389,5577,6] + - ["TR",1442084400000,170,5666,22] + - ["TR",1442080800000,-1,5931,38] + - ["TR",1442077200000,89,2968,24] + - ["TR",1442070000000,236,894,19] + - ["TR",1442066400000,85,1023,24] + - ["TR",1442062800000,315,1065,12] + - ["TR",1442055600000,299,1064,15] + - ["TR",1442052000000,41,869,28] + - ["TR",1442048400000,88,785,21] + - ["TR",1442044800000,41,776,24] + - ["TR",1442041200000,1,477,35] + - ["TR",1442023200000,306,436,13] + - ["TN",1442098800000,-9,-9,36] + - ["TJ",1442048400000,1471,1471,4] + - ["TH",1442084400000,13,-21,35] + - ["TH",1442070000000,0,-67,43] + - ["TH",1442066400000,-34,-67,45] + - ["TH",1442062800000,-46,-89,39] + - ["TH",1442055600000,0,8,37] + - ["TH",1442052000000,-22,11,46] + - ["TH",1442044800000,110,45,20] + - ["TH",1442041200000,3,91,33] + - ["TH",1442034000000,0,91,27] + - ["SV",1442088000000,9,114,29] + - ["SV",1442084400000,106,114,25] + - ["SV",1442019600000,-1,114,34] + - ["SK",1442098800000,7,361,25] + - ["SK",1442084400000,-92,367,50] + - ["SK",1442073600000,446,380,18] + - ["SK",1442062800000,6,379,31] + - ["SK",1442052000000,13,372,35] + - ["SK",1442037600000,-1,464,30] + - ["SI",1442091600000,9,-36,31] + - ["SI",1442080800000,-45,-36,46] + - ["SG",1442066400000,0,440,41] + - ["SG",1442062800000,388,517,11] + - ["SG",1442048400000,52,576,27] + - ["SG",1442044800000,77,579,21] + - ["SG",1442041200000,59,580,20] + - ["SG",1442037600000,3,2950,27] + - ["SG",1442030400000,1,2898,24] + - ["SG",1442026800000,2758,2821,3] + - ["SE",1442098800000,0,60,32] + - ["SE",1442095200000,61,97,20] + - ["SE",1442091600000,-1,186,35] + - ["SE",1442084400000,37,264,31] + - ["SE",1442080800000,89,278,15] + - ["SE",1442070000000,78,1693,26] + - ["SE",1442066400000,14,1689,32] + - ["SE",1442059200000,1476,1653,5] + - ["SE",1442055600000,-5,1419,41] + - ["SE",1442052000000,1,1432,36] + - ["SE",1442048400000,-145,1448,43] + - ["SE",1442041200000,91,-25,17] + - ["SE",1442030400000,30,89,20] + - ["SE",1442023200000,3,88,27] + - ["SE",1442019600000,109,233,14] + - ["SA",1442084400000,458,410,13] + - ["SA",1442077200000,-50,1686,42] + - ["SA",1442073600000,2,1686,33] + - ["SA",1442066400000,1276,1697,8] + - ["SA",1442059200000,0,1253,37] + - ["SA",1442055600000,11,1206,32] + - ["SA",1442048400000,14,1204,33] + - ["SA",1442037600000,-97,-72,34] + - ["RU",1442098800000,12098,15982,2] + - ["RU",1442095200000,435,18578,10] + - ["RU",1442091600000,3449,25039,3] + - ["RU",1442088000000,2596,25694,5] + - ["RU",1442084400000,6461,14758,3] + - ["RU",1442080800000,655,15941,7] + - ["RU",1442077200000,1162,17198,11] + - ["RU",1442073600000,1618,16649,10] + - ["RU",1442070000000,4706,10356,3] + - ["RU",1442066400000,2047,13603,5] + - ["RU",1442062800000,168,12940,16] + - ["RU",1442059200000,3902,12536,4] + - ["RU",1442055600000,499,8857,10] + - ["RU",1442052000000,1214,9374,8] + - ["RU",1442048400000,1027,9786,6] + - ["RU",1442044800000,2564,5560,3] + - ["RU",1442041200000,580,5719,8] + - ["RU",1442037600000,-324,4581,36] + - ["RU",1442034000000,658,3554,10] + - ["RU",1442030400000,76,1289,16] + - ["RU",1442026800000,0,2923,29] + - ["RU",1442023200000,299,3247,14] + - ["RU",1442019600000,2214,2589,2] + - ["RS",1442091600000,-15,74,39] + - ["RS",1442084400000,89,887,26] + - ["RS",1442080800000,0,887,37] + - ["RS",1442073600000,813,900,14] + - ["RS",1442066400000,0,921,40] + - ["RS",1442062800000,13,832,29] + - ["RS",1442019600000,6,832,27] + - ["RO",1442095200000,824,839,7] + - ["RO",1442091600000,0,810,34] + - ["RO",1442073600000,15,1351,31] + - ["RO",1442070000000,-29,1377,47] + - ["RO",1442062800000,541,872,10] + - ["RO",1442055600000,26,1156,28] + - ["RO",1442052000000,319,1986,16] + - ["RO",1442044800000,284,2083,14] + - ["RO",1442041200000,845,1542,6] + - ["RO",1442034000000,68,1516,20] + - ["QA",1442041200000,13,13,28] + - ["PY",1442084400000,628,634,10] + - ["PY",1442080800000,5,634,30] + - ["PY",1442019600000,1,634,29] + - ["PT",1442098800000,2,424,30] + - ["PT",1442095200000,19,345,24] + - ["PT",1442088000000,403,270,14] + - ["PT",1442080800000,-79,3740,47] + - ["PT",1442077200000,-75,3750,44] + - ["PT",1442070000000,3470,3833,5] + - ["PT",1442066400000,12,3441,33] + - ["PT",1442052000000,102,3692,22] + - ["PT",1442044800000,11,3767,30] + - ["PT",1442019600000,172,297,13] + - ["PR",1442095200000,29,-1,22] + - ["PR",1442077200000,5,1,32] + - ["PR",1442059200000,-35,23,41] + - ["PR",1442030400000,2,23,22] + - ["PR",1442026800000,22,-6,24] + - ["PL",1442098800000,-9,2744,35] + - ["PL",1442095200000,1851,3090,4] + - ["PL",1442091600000,902,3103,8] + - ["PL",1442088000000,346,3110,15] + - ["PL",1442084400000,13,3443,34] + - ["PL",1442080800000,7,1622,28] + - ["PL",1442077200000,324,866,19] + - ["PL",1442073600000,30,554,28] + - ["PL",1442070000000,146,4712,23] + - ["PL",1442066400000,34,4904,30] + - ["PL",1442062800000,4171,4990,4] + - ["PL",1442059200000,199,5290,21] + - ["PL",1442055600000,410,5510,14] + - ["PL",1442052000000,330,5795,15] + - ["PL",1442048400000,366,1905,13] + - ["PL",1442044800000,319,1801,12] + - ["PL",1442041200000,281,1391,10] + - ["PL",1442037600000,95,1061,14] + - ["PK",1442070000000,43,81,31] + - ["PK",1442062800000,23,105,24] + - ["PK",1442048400000,15,205,32] + - ["PK",1442041200000,24,306,25] + - ["PK",1442037600000,100,598,12] + - ["PK",1442026800000,101,575,18] + - ["PK",1442019600000,335,560,10] + - ["PH",1442098800000,8,863,24] + - ["PH",1442091600000,816,895,9] + - ["PH",1442084400000,39,897,30] + - ["PH",1442080800000,32,670,20] + - ["PH",1442077200000,2,696,33] + - ["PH",1442073600000,-227,1760,43] + - ["PH",1442070000000,34,1892,33] + - ["PH",1442066400000,1880,2133,6] + - ["PH",1442062800000,171,4100,15] + - ["PH",1442059200000,273,4349,17] + - ["PH",1442055600000,1969,4377,5] + - ["PH",1442052000000,22,2552,33] + - ["PH",1442048400000,62,2381,24] + - ["PH",1442044800000,55,2125,23] + - ["PH",1442041200000,0,215,36] + - ["PH",1442037600000,17,219,20] + - ["PH",1442034000000,59,1067,21] + - ["PH",1442030400000,26,1471,21] + - ["PH",1442026800000,910,1477,8] + - ["PH",1442023200000,459,1460,11] + - ["PH",1442019600000,6,1401,26] + - ["PE",1442098800000,1861,1774,4] + - ["PE",1442095200000,-19,1772,38] + - ["PE",1442084400000,-68,1609,49] + - ["PE",1442080800000,-2,1597,40] + - ["PE",1442077200000,-163,-276,45] + - ["PE",1442062800000,-12,-231,37] + - ["PE",1442026800000,-12,360,31] + - ["PE",1442023200000,26,362,23] + - ["PE",1442019600000,523,525,7] + - ["PA",1442026800000,0,0,28] + - ["OM",1442052000000,0,0,39] + - ["NZ",1442098800000,-2,399,33] + - ["NZ",1442095200000,-4,347,37] + - ["NZ",1442088000000,405,775,13] + - ["NZ",1442084400000,-52,964,48] + - ["NZ",1442059200000,428,1032,14] + - ["NZ",1442048400000,189,1671,17] + - ["NZ",1442037600000,66,1294,16] + - ["NZ",1442026800000,635,1346,11] + - ["NZ",1442019600000,28,918,24] + - ["NP",1442048400000,61,61,25] + - ["NO",1442098800000,2,16,29] + - ["NO",1442095200000,-1,31,34] + - ["NO",1442091600000,15,62,29] + - ["NO",1442088000000,15,284,27] + - ["NO",1442080800000,31,353,21] + - ["NO",1442073600000,222,383,20] + - ["NO",1442066400000,71,815,26] + - ["NO",1442055600000,29,353,27] + - ["NO",1442052000000,447,370,11] + - ["NO",1442048400000,-447,148,45] + - ["NO",1442019600000,48,77,20] + - ["NL",1442098800000,4,93,26] + - ["NL",1442095200000,70,105,18] + - ["NL",1442091600000,19,541,28] + - ["NL",1442088000000,12,9488,28] + - ["NL",1442084400000,436,10362,14] + - ["NL",1442080800000,8947,10458,2] + - ["NL",1442077200000,878,10355,13] + - ["NL",1442073600000,166,10404,22] + - ["NL",1442070000000,-84,9938,49] + - ["NL",1442066400000,61,1197,27] + - ["NL",1442062800000,-30,424,38] + - ["NL",1442059200000,206,311,20] + - ["NL",1442055600000,105,1698,21] + - ["NL",1442052000000,53,1653,24] + - ["NL",1442048400000,1303,1683,5] + - ["NL",1442044800000,16,1477,29] + - ["NL",1442034000000,0,1372,26] + - ["NG",1442070000000,6,214,39] + - ["NG",1442052000000,208,214,18] + - ["MY",1442098800000,739,730,5] + - ["MY",1442077200000,-10,731,38] + - ["MY",1442073600000,1,731,35] + - ["MY",1442066400000,1,732,35] + - ["MY",1442059200000,0,642,36] + - ["MY",1442055600000,1,525,35] + - ["MY",1442048400000,649,1459,10] + - ["MY",1442044800000,-127,2486,39] + - ["MY",1442041200000,935,2483,4] + - ["MY",1442034000000,1028,2475,6] + - ["MY",1442030400000,-3,1826,28] + - ["MY",1442019600000,-7,1953,35] + - ["MX",1442098800000,28,371,18] + - ["MX",1442095200000,-456,865,44] + - ["MX",1442091600000,799,1846,11] + - ["MX",1442088000000,494,1470,12] + - ["MX",1442084400000,981,5316,7] + - ["MX",1442080800000,-376,5751,49] + - ["MX",1442077200000,3874,4951,5] + - ["MX",1442073600000,-21,4456,40] + - ["MX",1442070000000,-1,3181,45] + - ["MX",1442066400000,-1,3561,42] + - ["MX",1442041200000,-294,631,43] + - ["MX",1442037600000,4,1025,24] + - ["MX",1442034000000,944,4668,8] + - ["MX",1442030400000,373,5218,12] + - ["MX",1442026800000,3642,5445,2] + - ["MX",1442023200000,549,5441,9] + - ["MX",1442016000000,-67,4497,13] + - ["MV",1442073600000,-3,-3,39] + - ["MT",1442048400000,-1,-1,38] + - ["MR",1442080800000,10,10,27] + - ["MO",1442070000000,18,48,36] + - ["MO",1442034000000,30,48,22] + - ["MM",1442073600000,25,28,29] + - ["MM",1442070000000,3,28,40] + - ["MK",1442077200000,-72,-72,43] + - ["MH",1442052000000,40,40,29] + - ["ME",1442073600000,0,0,37] + - ["MD",1442077200000,6916,6916,2] + - ["MA",1442098800000,8,263,23] + - ["MA",1442080800000,5,263,29] + - ["MA",1442077200000,250,207,20] + - ["MA",1442062800000,0,230,36] + - ["MA",1442059200000,-56,221,42] + - ["MA",1442055600000,23,216,29] + - ["MA",1442019600000,-1,-34,33] + - ["LV",1442095200000,0,0,32] + - ["LU",1442095200000,2,527,28] + - ["LU",1442077200000,525,606,15] + - ["LU",1442066400000,0,606,39] + - ["LU",1442059200000,79,606,23] + - ["LT",1442098800000,-24,-12,37] + - ["LT",1442080800000,12,-12,26] + - ["LK",1442084400000,-3,52,43] + - ["LK",1442052000000,47,131,26] + - ["LK",1442048400000,8,131,35] + - ["LK",1442026800000,79,131,20] + - ["LB",1442055600000,-67,-67,43] + - ["KZ",1442095200000,91,-248,16] + - ["KZ",1442084400000,-22,-248,47] + - ["KZ",1442077200000,-317,-248,48] + - ["KZ",1442066400000,0,-215,38] + - ["KZ",1442062800000,0,-243,35] + - ["KZ",1442059200000,33,191,27] + - ["KZ",1442055600000,63,947,23] + - ["KZ",1442052000000,412,1348,12] + - ["KZ",1442048400000,439,1509,12] + - ["KZ",1442044800000,401,1476,10] + - ["KZ",1442034000000,161,1413,17] + - ["KW",1442080800000,-33,1780,45] + - ["KW",1442077200000,-2,1778,36] + - ["KW",1442070000000,1815,1778,6] + - ["KW",1442055600000,-2,1778,40] + - ["KR",1442098800000,-36,1315,38] + - ["KR",1442095200000,827,1629,6] + - ["KR",1442088000000,524,1596,11] + - ["KR",1442084400000,314,1556,18] + - ["KR",1442080800000,-33,1814,44] + - ["KR",1442077200000,-40,4286,41] + - ["KR",1442070000000,222,4858,20] + - ["KR",1442066400000,3299,4752,3] + - ["KR",1442062800000,1096,8425,7] + - ["KR",1442059200000,208,8462,19] + - ["KR",1442055600000,3640,7866,4] + - ["KR",1442052000000,-3,5396,43] + - ["KR",1442048400000,-374,4320,44] + - ["KR",1442044800000,829,4138,8] + - ["KR",1442041200000,20,932,27] + - ["KR",1442037600000,26,1970,18] + - ["KR",1442034000000,434,2165,12] + - ["KR",1442030400000,1035,1655,7] + - ["KR",1442026800000,-179,2080,34] + - ["KR",1442023200000,319,3078,12] + - ["KR",1442019600000,445,2644,8] + - ["KR",1442016000000,1024,1609,2] + - ["KG",1442073600000,6,6,32] + - ["KE",1442044800000,-1,-1,34] + - ["JP",1442098800000,-6,222,34] + - ["JP",1442095200000,0,279,31] + - ["JP",1442091600000,228,292,18] + - ["JP",1442088000000,57,712,24] + - ["JP",1442084400000,13,769,33] + - ["JP",1442080800000,420,1931,10] + - ["JP",1442077200000,51,1782,27] + - ["JP",1442073600000,1162,1892,12] + - ["JP",1442070000000,79,2682,25] + - ["JP",1442066400000,167,2177,21] + - ["JP",1442062800000,803,3124,8] + - ["JP",1442059200000,-85,4751,43] + - ["JP",1442055600000,998,6653,8] + - ["JP",1442052000000,2789,8055,4] + - ["JP",1442048400000,1981,8625,3] + - ["JP",1442044800000,1569,10891,4] + - ["JP",1442041200000,1373,10803,3] + - ["JP",1442037600000,2181,8819,4] + - ["JP",1442034000000,910,7873,9] + - ["JP",1442030400000,805,8263,8] + - ["JP",1442026800000,1035,8892,6] + - ["JP",1442023200000,1959,6598,5] + - ["JP",1442019600000,2002,5688,4] + - ["JP",1442016000000,-113,4883,14] + - ["JO",1442080800000,4,2,32] + - ["JO",1442059200000,0,2,35] + - ["JO",1442055600000,-2,2,39] + - ["JM",1442070000000,30,30,35] + - ["IT",1442098800000,565,3856,7] + - ["IT",1442095200000,2940,7602,2] + - ["IT",1442091600000,351,10262,16] + - ["IT",1442088000000,3746,15806,4] + - ["IT",1442084400000,2660,17429,4] + - ["IT",1442080800000,5544,17075,3] + - ["IT",1442077200000,2188,16805,8] + - ["IT",1442073600000,2586,17214,7] + - ["IT",1442070000000,81,16492,24] + - ["IT",1442066400000,4155,11490,2] + - ["IT",1442062800000,1938,15542,5] + - ["IT",1442059200000,542,14836,12] + - ["IT",1442055600000,6240,15431,3] + - ["IT",1442052000000,1880,12759,6] + - ["IT",1442048400000,676,10841,9] + - ["IT",1442044800000,1483,10290,5] + - ["IT",1442041200000,20,5056,26] + - ["IT",1442037600000,-9,3159,32] + - ["IT",1442034000000,1006,2705,7] + - ["IT",1442030400000,-17,1333,29] + - ["IT",1442026800000,222,1496,13] + - ["IT",1442023200000,111,1505,17] + - ["IT",1442019600000,183,499,12] + - ["IT",1442016000000,0,516,9] + - ["IR",1442091600000,0,845,33] + - ["IR",1442088000000,714,811,10] + - ["IR",1442080800000,131,618,13] + - ["IR",1442077200000,-34,2073,40] + - ["IR",1442073600000,-193,1949,42] + - ["IR",1442059200000,1455,1390,7] + - ["IR",1442055600000,-124,1565,44] + - ["IR",1442052000000,155,1520,21] + - ["IR",1442044800000,306,1705,13] + - ["IR",1442041200000,-79,625,42] + - ["IR",1442034000000,-8,749,31] + - ["IR",1442030400000,375,594,10] + - ["IR",1442026800000,0,288,27] + - ["IQ",1442095200000,-2,4,35] + - ["IQ",1442052000000,0,3,38] + - ["IQ",1442044800000,6,3,31] + - ["IQ",1442041200000,-1,3,39] + - ["IN",1442095200000,4,312,27] + - ["IN",1442088000000,121,312,19] + - ["IN",1442084400000,187,6011,19] + - ["IN",1442080800000,0,7181,36] + - ["IN",1442077200000,5699,19268,4] + - ["IN",1442073600000,1170,19263,11] + - ["IN",1442070000000,12091,20623,2] + - ["IN",1442066400000,116,21331,23] + - ["IN",1442062800000,1547,15798,6] + - ["IN",1442059200000,708,15162,11] + - ["IN",1442055600000,166,3333,19] + - ["IN",1442052000000,534,5894,10] + - ["IN",1442048400000,262,4427,15] + - ["IN",1442044800000,2677,3854,2] + - ["IN",1442041200000,80,5038,18] + - ["IN",1442037600000,135,5952,11] + - ["IN",1442034000000,1350,6664,4] + - ["IN",1442030400000,1448,3845,6] + - ["IN",1442026800000,974,3803,7] + - ["IN",1442023200000,-142,3669,32] + - ["IN",1442019600000,38,2319,22] + - ["IN",1442016000000,1,871,7] + - ["IL",1442098800000,3,710,27] + - ["IL",1442095200000,0,1847,30] + - ["IL",1442091600000,707,1935,13] + - ["IL",1442084400000,1137,2122,6] + - ["IL",1442080800000,88,2150,17] + - ["IL",1442077200000,187,2199,22] + - ["IL",1442073600000,31,1495,27] + - ["IL",1442070000000,49,538,29] + - ["IL",1442066400000,3,1655,34] + - ["IL",1442062800000,180,1472,14] + - ["IL",1442059200000,1205,4186,8] + - ["IL",1442055600000,4,4162,33] + - ["IL",1442052000000,2745,4377,5] + - ["IL",1442048400000,25,4232,30] + - ["IL",1442044800000,218,3027,16] + - ["IL",1442041200000,35,3023,24] + - ["IE",1442091600000,-71,1151,40] + - ["IE",1442084400000,819,1051,9] + - ["IE",1442077200000,403,2113,18] + - ["IE",1442070000000,-100,2140,50] + - ["IE",1442066400000,1062,2212,9] + - ["IE",1442048400000,27,1394,29] + - ["IE",1442030400000,1,991,23] + - ["IE",1442026800000,1,1091,26] + - ["ID",1442098800000,13,54,19] + - ["ID",1442095200000,20,96,23] + - ["ID",1442091600000,21,113,26] + - ["ID",1442070000000,42,129,32] + - ["ID",1442059200000,17,-272,29] + - ["ID",1442055600000,16,-290,30] + - ["ID",1442044800000,-388,-297,41] + - ["ID",1442041200000,2,-320,34] + - ["ID",1442037600000,14,-58,22] + - ["ID",1442034000000,19,-490,24] + - ["ID",1442030400000,279,4,13] + - ["ID",1442026800000,-416,2,36] + - ["ID",1442023200000,106,-12,18] + - ["HU",1442098800000,110,376,13] + - ["HU",1442095200000,271,305,14] + - ["HU",1442091600000,-5,446,38] + - ["HU",1442088000000,-71,688,34] + - ["HU",1442084400000,141,628,23] + - ["HU",1442080800000,242,355,11] + - ["HU",1442062800000,50,859,21] + - ["HU",1442055600000,-2,1477,38] + - ["HU",1442048400000,499,1427,11] + - ["HU",1442044800000,547,1382,9] + - ["HU",1442041200000,91,1378,16] + - ["HU",1442037600000,197,1380,10] + - ["HU",1442019600000,46,881,21] + - ["HR",1442088000000,82,292,20] + - ["HR",1442084400000,-10,350,45] + - ["HR",1442080800000,220,350,12] + - ["HR",1442077200000,58,382,26] + - ["HR",1442073600000,0,300,36] + - ["HR",1442070000000,32,310,34] + - ["HN",1442026800000,-1,-1,30] + - ["HK",1442095200000,-1,-4,33] + - ["HK",1442091600000,-3,5541,37] + - ["HK",1442080800000,0,5855,35] + - ["HK",1442073600000,5545,5894,4] + - ["HK",1442070000000,314,5896,16] + - ["HK",1442066400000,39,5901,28] + - ["HK",1442062800000,1,6705,33] + - ["HK",1442059200000,2,1175,33] + - ["HK",1442055600000,804,862,9] + - ["HK",1442052000000,15,844,34] + - ["HK",1442048400000,1,828,37] + - ["HK",1442044800000,21,1462,28] + - ["HK",1442041200000,-15,1795,40] + - ["HK",1442037600000,636,1937,5] + - ["HK",1442034000000,1137,1725,5] + - ["HK",1442030400000,157,4118,14] + - ["HK",1442026800000,-211,4020,35] + - ["HK",1442023200000,2414,3384,4] + - ["HK",1442019600000,-113,2247,36] + - ["GT",1442098800000,1,7,31] + - ["GT",1442026800000,173,7,15] + - ["GT",1442023200000,-167,7,33] + - ["GR",1442091600000,123,390,20] + - ["GR",1442084400000,179,76,21] + - ["GR",1442080800000,88,78,16] + - ["GR",1442073600000,-314,86,44] + - ["GR",1442070000000,2,-63,41] + - ["GR",1442062800000,8,-235,30] + - ["GR",1442048400000,-26,-323,42] + - ["GR",1442041200000,7,73,30] + - ["GR",1442034000000,0,71,25] + - ["GR",1442019600000,82,63,16] + - ["GH",1442088000000,0,0,32] + - ["GE",1442080800000,-27,-119,42] + - ["GE",1442062800000,16,-140,28] + - ["GE",1442052000000,-108,-140,47] + - ["GE",1442044800000,-21,-140,37] + - ["GB",1442098800000,49,1098,16] + - ["GB",1442095200000,238,2691,15] + - ["GB",1442091600000,811,3075,10] + - ["GB",1442088000000,1593,4519,9] + - ["GB",1442084400000,384,5605,16] + - ["GB",1442080800000,1444,6015,5] + - ["GB",1442077200000,1135,5578,12] + - ["GB",1442073600000,648,4656,16] + - ["GB",1442070000000,374,10015,15] + - ["GB",1442066400000,671,24682,13] + - ["GB",1442062800000,5743,24000,2] + - ["GB",1442059200000,16111,23520,2] + - ["GB",1442055600000,453,23886,13] + - ["GB",1442052000000,168,23247,20] + - ["GB",1442048400000,740,17546,8] + - ["GB",1442044800000,32,1979,26] + - ["GB",1442041200000,42,1514,22] + - ["GB",1442037600000,544,3870,7] + - ["GB",1442034000000,-12,3469,32] + - ["GB",1442030400000,2524,5253,2] + - ["GB",1442026800000,339,5265,12] + - ["GB",1442023200000,1816,4677,6] + - ["GB",1442019600000,54,4689,19] + - ["GB",1442016000000,-44,2165,12] + - ["FR",1442098800000,136,964,12] + - ["FR",1442095200000,87,2836,17] + - ["FR",1442091600000,741,9479,12] + - ["FR",1442088000000,1872,10036,7] + - ["FR",1442084400000,6643,9456,2] + - ["FR",1442080800000,557,12891,8] + - ["FR",1442077200000,-444,12624,50] + - ["FR",1442073600000,3522,13268,6] + - ["FR",1442070000000,474,12301,12] + - ["FR",1442066400000,2516,16394,4] + - ["FR",1442062800000,5676,17301,3] + - ["FR",1442059200000,4650,14416,3] + - ["FR",1442055600000,463,16969,12] + - ["FR",1442052000000,637,14625,9] + - ["FR",1442048400000,3027,9553,2] + - ["FR",1442044800000,172,9077,18] + - ["FR",1442041200000,604,9090,7] + - ["FR",1442037600000,4174,8539,2] + - ["FR",1442034000000,476,6140,11] + - ["FR",1442026800000,86,6553,19] + - ["FR",1442023200000,628,5948,8] + - ["FR",1442019600000,585,1774,6] + - ["FR",1442016000000,-1,1298,11] + - ["FI",1442095200000,69,1068,19] + - ["FI",1442084400000,895,1268,8] + - ["FI",1442080800000,104,1267,14] + - ["FI",1442077200000,200,1450,21] + - ["FI",1442073600000,-1,1400,38] + - ["FI",1442066400000,183,912,20] + - ["FI",1442062800000,19,994,26] + - ["FI",1442059200000,407,806,15] + - ["FI",1442052000000,186,821,19] + - ["FI",1442048400000,12,2129,34] + - ["FI",1442037600000,14,2110,21] + - ["FI",1442030400000,1491,1703,5] + - ["ES",1442098800000,458,3664,9] + - ["ES",1442095200000,2506,3534,3] + - ["ES",1442091600000,700,3871,14] + - ["ES",1442088000000,-130,5111,35] + - ["ES",1442084400000,337,4807,17] + - ["ES",1442077200000,1240,2362,10] + - ["ES",1442073600000,154,2123,23] + - ["ES",1442070000000,61,2182,27] + - ["ES",1442066400000,461,2931,15] + - ["ES",1442062800000,-71,2186,40] + - ["ES",1442059200000,1086,2028,9] + - ["ES",1442055600000,495,2125,11] + - ["ES",1442052000000,-4,1495,44] + - ["ES",1442048400000,158,1684,19] + - ["ES",1442044800000,-169,601,40] + - ["ES",1442041200000,118,54,15] + - ["ES",1442037600000,3,53,26] + - ["ES",1442034000000,-52,-2,34] + - ["ES",1442023200000,-5,167,28] + - ["ES",1442019600000,103,49,15] + - ["EG",1442091600000,27,140,25] + - ["EG",1442073600000,1,154,34] + - ["EG",1442062800000,112,170,18] + - ["EG",1442055600000,14,170,31] + - ["EG",1442026800000,16,143,25] + - ["EE",1442044800000,-19,18,36] + - ["EE",1442041200000,37,18,23] + - ["EC",1442095200000,10,212,26] + - ["EC",1442084400000,568,212,11] + - ["EC",1442077200000,-366,203,49] + - ["EC",1442030400000,0,232,25] + - ["EC",1442023200000,-9,222,29] + - ["EC",1442019600000,29,-346,23] + - ["DZ",1442077200000,-1,-1,35] + - ["DO",1442095200000,13,221,25] + - ["DO",1442084400000,8,256,36] + - ["DO",1442073600000,200,264,21] + - ["DO",1442066400000,35,264,29] + - ["DO",1442023200000,8,251,26] + - ["DK",1442095200000,0,42,29] + - ["DK",1442091600000,139,103,19] + - ["DK",1442084400000,-97,94,51] + - ["DK",1442080800000,61,510,18] + - ["DK",1442077200000,-9,511,37] + - ["DK",1442066400000,416,372,16] + - ["DK",1442062800000,1,511,32] + - ["DK",1442059200000,0,445,34] + - ["DK",1442055600000,42,490,25] + - ["DK",1442048400000,-5,84,39] + - ["DK",1442044800000,36,83,25] + - ["DK",1442037600000,10,83,23] + - ["DE",1442098800000,329,5691,10] + - ["DE",1442095200000,1007,5881,5] + - ["DE",1442091600000,4355,5756,2] + - ["DE",1442088000000,190,6889,18] + - ["DE",1442084400000,-125,8748,52] + - ["DE",1442080800000,1133,13816,6] + - ["DE",1442077200000,2188,11127,7] + - ["DE",1442073600000,6075,12514,3] + - ["DE",1442070000000,1666,12922,8] + - ["DE",1442066400000,1577,12078,7] + - ["DE",1442062800000,283,11413,13] + - ["DE",1442059200000,289,6938,16] + - ["DE",1442055600000,1523,6083,6] + - ["DE",1442052000000,1600,5485,7] + - ["DE",1442048400000,811,5399,7] + - ["DE",1442044800000,979,5654,7] + - ["DE",1442041200000,197,4489,12] + - ["DE",1442037600000,544,3262,6] + - ["DE",1442034000000,358,2515,13] + - ["DE",1442030400000,373,1536,11] + - ["DE",1442023200000,64,1506,20] + - ["DE",1442019600000,0,962,31] + - ["DE",1442016000000,167,604,3] + - ["CZ",1442098800000,2,89,28] + - ["CZ",1442080800000,-28,2140,43] + - ["CZ",1442077200000,115,2308,23] + - ["CZ",1442073600000,2051,2308,9] + - ["CZ",1442070000000,168,2327,22] + - ["CZ",1442062800000,0,3428,34] + - ["CZ",1442059200000,21,3331,28] + - ["CZ",1442055600000,1073,1358,7] + - ["CZ",1442037600000,18,1171,19] + - ["CZ",1442034000000,78,1171,19] + - ["CZ",1442026800000,-19,1150,33] + - ["CR",1442088000000,72,429,21] + - ["CR",1442048400000,163,480,18] + - ["CR",1442044800000,194,2977,17] + - ["CR",1442041200000,51,3117,21] + - ["CR",1442030400000,2497,3107,3] + - ["CR",1442026800000,140,3006,17] + - ["CR",1442023200000,62,2812,21] + - ["CR",1442019600000,62,2761,18] + - ["CO",1442098800000,83,750,14] + - ["CO",1442095200000,290,17900,13] + - ["CO",1442091600000,377,17951,15] + - ["CO",1442088000000,17150,17976,2] + - ["CO",1442084400000,51,18474,28] + - ["CO",1442080800000,25,58044,23] + - ["CO",1442077200000,581,57622,14] + - ["CO",1442073600000,39860,40760,2] + - ["CO",1442070000000,-45,41182,48] + - ["CO",1442066400000,288,41598,19] + - ["CO",1442059200000,473,41026,13] + - ["CO",1442030400000,441,1178,9] + - ["CO",1442023200000,9,1239,25] + - ["CO",1442019600000,12,951,25] + - ["CO",1442016000000,16,478,6] + - ["CN",1442084400000,-1,-26,41] + - ["CN",1442080800000,-10,-18,41] + - ["CN",1442066400000,-15,51,43] + - ["CN",1442059200000,8,51,31] + - ["CN",1442055600000,69,345,22] + - ["CN",1442052000000,0,453,37] + - ["CN",1442048400000,293,622,14] + - ["CN",1442037600000,98,601,13] + - ["CN",1442026800000,154,532,16] + - ["CN",1442023200000,-13,532,31] + - ["CL",1442098800000,9,-247,21] + - ["CL",1442095200000,-276,39,42] + - ["CL",1442091600000,20,165,27] + - ["CL",1442088000000,286,182,16] + - ["CL",1442084400000,126,158,24] + - ["CL",1442080800000,17,587,25] + - ["CL",1442077200000,-15,580,39] + - ["CL",1442073600000,153,253,24] + - ["CL",1442070000000,13,144,37] + - ["CL",1442066400000,-41,115,46] + - ["CL",1442062800000,17,520,27] + - ["CL",1442059200000,-12,366,40] + - ["CL",1442052000000,390,355,14] + - ["CL",1442041200000,-1,395,38] + - ["CL",1442037600000,2,418,28] + - ["CL",1442034000000,-1,445,28] + - ["CL",1442030400000,40,-315,18] + - ["CL",1442023200000,15,-153,24] + - ["CL",1442019600000,-370,-155,37] + - ["CL",1442016000000,161,-154,4] + - ["CH",1442091600000,67,86,23] + - ["CH",1442084400000,13,446,32] + - ["CH",1442077200000,6,457,31] + - ["CH",1442073600000,360,479,19] + - ["CH",1442070000000,11,412,38] + - ["CH",1442062800000,22,446,25] + - ["CH",1442055600000,0,464,36] + - ["CH",1442052000000,47,50,25] + - ["CH",1442048400000,24,237,31] + - ["CH",1442044800000,-54,274,38] + - ["CH",1442041200000,198,274,11] + - ["CH",1442037600000,59,227,17] + - ["CA",1442098800000,164,2661,11] + - ["CA",1442095200000,-361,2696,43] + - ["CA",1442091600000,2858,2740,5] + - ["CA",1442088000000,35,3221,25] + - ["CA",1442084400000,44,2775,29] + - ["CA",1442080800000,481,3222,9] + - ["CA",1442077200000,-282,549,47] + - ["CA",1442073600000,86,821,26] + - ["CA",1442070000000,185,410,21] + - ["CA",1442066400000,307,965,17] + - ["CA",1442062800000,-367,1285,44] + - ["CA",1442059200000,1036,1200,10] + - ["CA",1442052000000,38,1020,31] + - ["CA",1442044800000,1,581,32] + - ["CA",1442041200000,5,1126,31] + - ["CA",1442037600000,-132,43,35] + - ["CA",1442034000000,178,2221,16] + - ["CA",1442030400000,-47,2506,30] + - ["CA",1442026800000,2216,4685,4] + - ["CA",1442023200000,286,4446,15] + - ["CA",1442019600000,2184,4268,3] + - ["CA",1442016000000,-371,4315,16] + - ["BY",1442088000000,33,62,26] + - ["BY",1442084400000,1,92,38] + - ["BY",1442080800000,28,688,22] + - ["BY",1442077200000,30,2152,28] + - ["BY",1442073600000,596,2120,17] + - ["BY",1442059200000,1464,2119,6] + - ["BY",1442055600000,1,2091,34] + - ["BR",1442098800000,-645,114,40] + - ["BR",1442095200000,748,329,8] + - ["BR",1442091600000,11,821,30] + - ["BR",1442088000000,215,554,17] + - ["BR",1442084400000,492,3452,12] + - ["BR",1442080800000,-267,4791,48] + - ["BR",1442077200000,2253,5316,6] + - ["BR",1442073600000,2087,6135,8] + - ["BR",1442070000000,536,5736,9] + - ["BR",1442066400000,1034,6076,10] + - ["BR",1442062800000,93,4065,19] + - ["BR",1442059200000,73,2210,24] + - ["BR",1442055600000,242,1745,16] + - ["BR",1442052000000,232,714,17] + - ["BR",1442044800000,71,888,22] + - ["BR",1442041200000,3,836,32] + - ["BR",1442037600000,267,624,8] + - ["BR",1442034000000,21,443,23] + - ["BR",1442030400000,30,1251,19] + - ["BR",1442026800000,51,1620,23] + - ["BR",1442023200000,879,1105,7] + - ["BR",1442019600000,372,1084,9] + - ["BR",1442016000000,-248,1054,15] + - ["BO",1442095200000,-4,4,36] + - ["BO",1442088000000,4,4,31] + - ["BO",1442080800000,4,4,31] + - ["BH",1442052000000,44,44,27] + - ["BG",1442084400000,401,647,15] + - ["BG",1442070000000,55,19583,28] + - ["BG",1442059200000,191,19592,22] + - ["BG",1442052000000,18936,19592,2] + - ["BG",1442041200000,9,19191,29] + - ["BE",1442098800000,9,177,20] + - ["BE",1442091600000,101,360,21] + - ["BE",1442088000000,67,361,22] + - ["BE",1442084400000,183,380,20] + - ["BE",1442080800000,1,507,34] + - ["BE",1442073600000,19,497,30] + - ["BE",1442066400000,136,663,22] + - ["BE",1442062800000,91,479,20] + - ["BE",1442055600000,233,537,17] + - ["BE",1442052000000,-1,415,40] + - ["BE",1442048400000,59,279,26] + - ["BE",1442030400000,-103,188,31] + - ["BD",1442091600000,-2,176,36] + - ["BD",1442077200000,75,252,25] + - ["BD",1442073600000,103,1106,25] + - ["BD",1442066400000,76,1106,25] + - ["BD",1442041200000,854,1108,5] + - ["BD",1442019600000,0,1033,30] + - ["BA",1442084400000,-1,-165,40] + - ["BA",1442055600000,-202,-178,45] + - ["BA",1442052000000,38,-178,30] + - ["BA",1442048400000,-13,-178,40] + - ["AU",1442098800000,518,2051,8] + - ["AU",1442095200000,395,2052,12] + - ["AU",1442091600000,1138,2040,7] + - ["AU",1442077200000,1,2019,34] + - ["AU",1442070000000,-12,1539,46] + - ["AU",1442066400000,-21,1326,44] + - ["AU",1442059200000,38,-455,26] + - ["AU",1442055600000,182,-321,18] + - ["AU",1442052000000,-643,64,48] + - ["AU",1442048400000,135,279,20] + - ["AU",1442044800000,373,244,11] + - ["AU",1442041200000,194,345,13] + - ["AU",1442037600000,3,611,25] + - ["AU",1442034000000,283,664,14] + - ["AU",1442030400000,-377,343,32] + - ["AU",1442026800000,188,402,14] + - ["AU",1442023200000,52,399,22] + - ["AU",1442019600000,253,116,11] + - ["AU",1442016000000,0,493,8] + - ["AT",1442091600000,89,7137,22] + - ["AT",1442088000000,7050,7409,3] + - ["AT",1442084400000,-2,7409,42] + - ["AT",1442070000000,272,7254,18] + - ["AT",1442066400000,0,11958,37] + - ["AT",1442062800000,-155,4908,41] + - ["AT",1442052000000,4793,4910,3] + - ["AR",1442098800000,64,1034,15] + - ["AR",1442095200000,630,1029,9] + - ["AR",1442091600000,340,1030,17] + - ["AR",1442084400000,-5,439,44] + - ["AR",1442080800000,1,752,33] + - ["AR",1442077200000,-591,122,51] + - ["AR",1442070000000,377,-189,14] + - ["AR",1442066400000,0,26,36] + - ["AR",1442062800000,29,-29,23] + - ["AR",1442059200000,210,643,18] + - ["AR",1442055600000,-54,478,42] + - ["AR",1442037600000,81,475,15] + - ["AR",1442034000000,212,1090,15] + - ["AR",1442030400000,-3,3394,27] + - ["AR",1442026800000,644,3449,10] + - ["AR",1442023200000,2514,3368,3] + - ["AR",1442019600000,1,3156,28] + - ["AO",1442098800000,722,766,6] + - ["AO",1442088000000,62,740,23] + - ["AO",1442052000000,-18,740,45] + - ["AO",1442041200000,-26,740,41] + - ["AL",1442091600000,54,80,24] + - ["AL",1442077200000,26,80,29] + - ["AE",1442080800000,42,6411,19] + - ["AE",1442077200000,6323,6400,3] + - ["AE",1442070000000,46,6397,30] + - ["AE",1442059200000,-11,6436,39] + - ["AE",1442052000000,-3,6387,42] + - ["AE",1442048400000,39,182,28] + - ["AE",1442044800000,-7,136,35] + - ["AE",1442030400000,118,147,15] + - [null,1442098800000,276159,1165087,1] + - [null,1442095200000,374501,1468959,1] + - [null,1442091600000,514427,1841528,1] + - [null,1442088000000,303872,2294605,1] + - [null,1442084400000,372569,2410929,1] + - [null,1442080800000,453077,2411822,1] + - [null,1442077200000,392483,2255910,1] + - [null,1442073600000,375394,2303622,1] + - [null,1442070000000,358515,2320518,1] + - [null,1442066400000,351584,2326738,1] + - [null,1442062800000,389465,2218213,1] + - [null,1442059200000,459297,2630189,1] + - [null,1442055600000,283958,2579990,1] + - [null,1442052000000,787370,2544408,1] + - [null,1442048400000,308316,2698393,1] + - [null,1442044800000,316002,2439701,1] + - [null,1442041200000,543450,2486700,1] + - [null,1442037600000,200605,1866002,1] + - [null,1442034000000,330957,1810312,1] + - [null,1442030400000,166672,1893946,1] + - [null,1442026800000,252626,1524388,1] + - [null,1442023200000,399636,1353656,1] + - [null,1442019600000,173892,1022699,1] + - [null,1442016000000,29873,856027,1] From 039b05585cb0984fa773393b9bf3b303796513af Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 30 Oct 2023 09:43:22 +0530 Subject: [PATCH 11/15] Add worker status and duration metrics in live and task reports (#15180) Add worker status and duration metrics in live and task reports for tracking. --- docs/api-reference/sql-ingestion-api.md | 5 ++ .../apache/druid/msq/exec/ControllerImpl.java | 3 + .../msq/indexing/MSQWorkerTaskLauncher.java | 77 ++++++++++++++++++- .../msq/indexing/report/MSQStatusReport.java | 13 ++++ .../indexing/report/MSQTaskReportTest.java | 7 +- .../resources/SqlStatementResourceTest.java | 2 + .../util/SqlStatementResourceHelperTest.java | 5 ++ 7 files changed, 105 insertions(+), 7 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 3daadfa5085f..5492c3ea46dd 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -603,6 +603,11 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.status` | RUNNING, SUCCESS, or FAILED. | | `multiStageQuery.payload.status.startTime` | Start time of the query in ISO format. Only present if the query has started running. | | `multiStageQuery.payload.status.durationMs` | Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet. | +| `multiStageQuery.payload.status.workers` | Workers for the controller task.| +| `multiStageQuery.payload.status.workers.` | Array of worker tasks including retries. | +| `multiStageQuery.payload.status.workers.[].workerId` | Id of the worker task.| | +| `multiStageQuery.payload.status.workers.[].status` | RUNNING, SUCCESS, or FAILED.| +| `multiStageQuery.payload.status.workers.[].durationMs` | Milliseconds elapsed after the worker task started running. It is -1 for worker tasks with status RUNNING.| | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | | `multiStageQuery.payload.status.segmentLoadStatus` | Segment loading container. Only present after the segments have been published. | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index f2260b055a96..1ae3cdc5a6e1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -2262,11 +2262,13 @@ private static MSQStatusReport makeStatusReport( { int pendingTasks = -1; int runningTasks = 1; + Map> workerStatsMap = new HashMap<>(); if (taskLauncher != null) { WorkerCount workerTaskCount = taskLauncher.getWorkerTaskCount(); pendingTasks = workerTaskCount.getPendingWorkerCount(); runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller. + workerStatsMap = taskLauncher.getWorkerStats(); } SegmentLoadStatusFetcher.SegmentLoadWaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); @@ -2277,6 +2279,7 @@ private static MSQStatusReport makeStatusReport( errorReports, queryStartTime, queryDuration, + workerStatsMap, pendingTasks, runningTasks, status diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index dcc81d868644..c2092e7f24ac 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.indexing; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; @@ -47,15 +48,18 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.OptionalLong; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -108,10 +112,11 @@ private enum State @GuardedBy("taskIds") private final IntSet fullyStartedTasks = new IntOpenHashSet(); - // Mutable state accessible only to the main loop. LinkedHashMap since order of key set matters. Tasks are added - // here once they are submitted for running, but before they are fully started up. + // Mutable state accessed by mainLoop, ControllerImpl, and jetty (/liveReports) threads. + // Tasks are added here once they are submitted for running, but before they are fully started up. // taskId -> taskTracker - private final Map taskTrackers = new LinkedHashMap<>(); + private final ConcurrentMap taskTrackers = new ConcurrentSkipListMap<>(Comparator.comparingInt( + MSQTasks::workerFromTaskId)); // Set of tasks which are issued a cancel request by the controller. private final Set canceledWorkerTasks = ConcurrentHashMap.newKeySet(); @@ -348,6 +353,70 @@ public boolean isTaskLatest(String taskId) } } + public static class WorkerStats + { + String workerId; + TaskState state; + long duration; + + /** + * For JSON deserialization only + */ + public WorkerStats() + { + } + + public WorkerStats(String workerId, TaskState state, long duration) + { + this.workerId = workerId; + this.state = state; + this.duration = duration; + } + + @JsonProperty + public String getWorkerId() + { + return workerId; + } + + @JsonProperty + public TaskState getState() + { + return state; + } + + @JsonProperty("durationMs") + public long getDuration() + { + return duration; + } + } + + public Map> getWorkerStats() + { + final Map> workerStats = new TreeMap<>(); + + for (Map.Entry taskEntry : taskTrackers.entrySet()) { + + TaskTracker taskTracker = taskEntry.getValue(); + + workerStats.computeIfAbsent(taskTracker.workerNumber, k -> new ArrayList<>()) + .add(new WorkerStats(taskEntry.getKey(), + taskTracker.status.getStatusCode(), + // getDuration() returns -1 for running tasks. + // It's not calculated on-the-fly here since + // taskTracker.startTimeMillis marks task + // submission time rather than the actual start. + taskTracker.status.getDuration() + )); + } + + for (List workerStatsList : workerStats.values()) { + workerStatsList.sort(Comparator.comparing(WorkerStats::getWorkerId)); + } + return workerStats; + } + private void mainLoop() { try { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java index d3864498349d..1dd7d6589031 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -25,12 +25,15 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexer.TaskState; import org.apache.druid.msq.exec.SegmentLoadStatusFetcher; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Collection; import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Objects; public class MSQStatusReport @@ -47,6 +50,8 @@ public class MSQStatusReport private final long durationMs; + private final Map> workerStats; + private final int pendingTasks; private final int runningTasks; @@ -61,6 +66,7 @@ public MSQStatusReport( @JsonProperty("warnings") Collection warningReports, @JsonProperty("startTime") @Nullable DateTime startTime, @JsonProperty("durationMs") long durationMs, + @JsonProperty("workers") Map> workerStats, @JsonProperty("pendingTasks") int pendingTasks, @JsonProperty("runningTasks") int runningTasks, @JsonProperty("segmentLoadWaiterStatus") @Nullable SegmentLoadStatusFetcher.SegmentLoadWaiterStatus segmentLoadWaiterStatus @@ -71,6 +77,7 @@ public MSQStatusReport( this.warningReports = warningReports != null ? warningReports : Collections.emptyList(); this.startTime = startTime; this.durationMs = durationMs; + this.workerStats = workerStats; this.pendingTasks = pendingTasks; this.runningTasks = runningTasks; this.segmentLoadWaiterStatus = segmentLoadWaiterStatus; @@ -123,6 +130,12 @@ public long getDurationMs() return durationMs; } + @JsonProperty("workers") + public Map> getWorkerStats() + { + return workerStats; + } + @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index 3b49572996ec..4bc3d1075c10 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -55,6 +55,7 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -107,7 +108,7 @@ public void testSerdeResultsReport() throws Exception final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2, status), + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, new HashMap<>(), 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), @@ -172,7 +173,7 @@ public void testSerdeErrorReport() throws Exception final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.FAILED, errorReport, new ArrayDeque<>(), null, 0, 1, 2, status), + new MSQStatusReport(TaskState.FAILED, errorReport, new ArrayDeque<>(), null, 0, new HashMap<>(), 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), @@ -220,7 +221,7 @@ public void testWriteTaskReport() throws Exception final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2, status), + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, new HashMap<>(), 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index d6572801207f..e3995a4a96c8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -244,6 +244,7 @@ public class SqlStatementResourceTest extends MSQTestBase new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null @@ -310,6 +311,7 @@ public class SqlStatementResourceTest extends MSQTestBase new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/SqlStatementResourceHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/SqlStatementResourceHelperTest.java index 806bd8ebe988..0254a61a2c71 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/SqlStatementResourceHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/SqlStatementResourceHelperTest.java @@ -67,6 +67,7 @@ public void testDistinctPartitionsOnEachWorker() new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null @@ -105,6 +106,7 @@ public void testOnePartitionOnEachWorker() new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null @@ -144,6 +146,7 @@ public void testCommonPartitionsOnEachWorker() new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null @@ -181,6 +184,7 @@ public void testNullChannelCounters() new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null @@ -220,6 +224,7 @@ public void testConsecutivePartitionsOnEachWorker() new ArrayDeque<>(), null, 0, + new HashMap<>(), 1, 2, null From 275c1ec64c622164489bcd34971cc85beadebedc Mon Sep 17 00:00:00 2001 From: Ben Sykes Date: Sun, 29 Oct 2023 21:22:52 -0700 Subject: [PATCH 12/15] Fix error assuming a Complex Type that is a Number is a double (#15272) * Fix error assuming a Complex Type that is a Number is a double In the case where a complex type is a number, it may not be castable to double. It can safely be case as Number first to get to the doubleValue. --- .../filter/PredicateValueMatcherFactory.java | 2 +- .../PredicateValueMatcherFactoryTest.java | 76 +++++++++++++++++++ 2 files changed, 77 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java index 796177a4c79d..0709f8e18367 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java @@ -189,7 +189,7 @@ public boolean matches(boolean includeUnknown) return getFloatPredicate().applyFloat((float) rowValue); } else if (rowValue instanceof Number) { // Double or some other non-int, non-long, non-float number. - return getDoublePredicate().applyDouble((double) rowValue); + return getDoublePredicate().applyDouble(((Number) rowValue).doubleValue()); } else if (rowValue instanceof Object[]) { return getArrayPredicate().apply((Object[]) rowValue); } else { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java index 30ec09d0d3e1..c2e7a4912575 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java @@ -157,6 +157,82 @@ public void testDoubleProcessorNotMatchingValue() Assert.assertFalse(matcher.matches(false)); } + @Test + public void testNumberProcessorMatchingValue() + { + Double num = 2.; + final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( + Number.class, + ImmutableList.of(new Number() { + @Override + public int intValue() + { + return num.intValue(); + } + + @Override + public long longValue() + { + return num.longValue(); + } + + @Override + public float floatValue() + { + return num.floatValue(); + } + + @Override + public double doubleValue() + { + return num; + } + }), + DateTimes.nowUtc() + ); + columnValueSelector.advance(); + final ValueMatcher matcher = forSelector("2").makeComplexProcessor(columnValueSelector); + Assert.assertTrue(matcher.matches(false)); + } + + @Test + public void testNumberProcessorNotMatchingValue() + { + Double num = 2.; + final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( + Double.class, + ImmutableList.of(new Number() { + @Override + public int intValue() + { + return num.intValue(); + } + + @Override + public long longValue() + { + return num.longValue(); + } + + @Override + public float floatValue() + { + return num.floatValue(); + } + + @Override + public double doubleValue() + { + return num; + } + }), + DateTimes.nowUtc() + ); + columnValueSelector.advance(); + final ValueMatcher matcher = forSelector("5").makeComplexProcessor(columnValueSelector); + Assert.assertFalse(matcher.matches(false)); + } + @Test public void testLongProcessorMatchingValue() { From a27598a487a9edbe0ce7d094ccaf1bf515958a2b Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 30 Oct 2023 14:39:15 +0530 Subject: [PATCH 13/15] Segregate advance and advanceUninterruptibly flow in postJoinCursor to allow for interrupts in advance (#15222) Currently advance function in postJoinCursor calls advanceUninterruptibly which in turn keeps calling baseCursor.advanceUninterruptibly until the post join condition matches, without checking for interrupts. This causes the CPU to hit 100% without getting a chance for query to be cancelled. With this change, the call flow of advance and advanceUninterruptibly is separated out so that they call baseCursor.advance and baseCursor.advanceUninterruptibly in them, respectively, giving a chance for interrupts in the former case between successive calls to baseCursor.advance. --- .../druid/segment/join/PostJoinCursor.java | 33 ++- .../segment/join/PostJoinCursorTest.java | 264 ++++++++++++++++++ 2 files changed, 292 insertions(+), 5 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 57da128c73d7..26c119bd34f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -19,7 +19,7 @@ package org.apache.druid.segment.join; -import org.apache.druid.query.BaseQuery; +import com.google.common.annotations.VisibleForTesting; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; @@ -39,7 +39,7 @@ public class PostJoinCursor implements Cursor private final ColumnSelectorFactory columnSelectorFactory; @Nullable - private final ValueMatcher valueMatcher; + private ValueMatcher valueMatcher; @Nullable private final Filter postJoinFilter; @@ -69,7 +69,28 @@ public static PostJoinCursor wrap( return postJoinCursor; } + @VisibleForTesting + public void setValueMatcher(@Nullable ValueMatcher valueMatcher) + { + this.valueMatcher = valueMatcher; + } + private void advanceToMatch() + { + if (valueMatcher != null) { + while (!isDone() && !valueMatcher.matches(false)) { + baseCursor.advance(); + } + } + } + + /** + * Matches tuples coming out of a join to a post-join condition uninterruptibly, and hence can be a long-running call. + * For this reason, {@link PostJoinCursor#advance()} instead calls {@link PostJoinCursor#advanceToMatch()} (unlike + * other cursors) that allows interruptions, thereby resolving issues where the + * CPU thread running PostJoinCursor cannot be terminated + */ + private void advanceToMatchUninterruptibly() { if (valueMatcher != null) { while (!isDone() && !valueMatcher.matches(false)) { @@ -99,15 +120,17 @@ public Filter getPostJoinFilter() @Override public void advance() { - advanceUninterruptibly(); - BaseQuery.checkInterrupted(); + baseCursor.advance(); + // Relies on baseCursor.advance() call inside this for BaseQuery.checkInterrupted() checks -- unlike other cursors + // which call advanceInterruptibly() and hence have to explicitly provision for interrupts. + advanceToMatch(); } @Override public void advanceUninterruptibly() { baseCursor.advanceUninterruptibly(); - advanceToMatch(); + advanceToMatchUninterruptibly(); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java new file mode 100644 index 000000000000..6813c04bb10f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -0,0 +1,264 @@ +/* + * 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.segment.join; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static java.lang.Thread.sleep; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest +{ + + public QueryableIndexSegment infiniteFactSegment; + + /** + * Simulates infinite segment by using a base cursor with advance() and advanceInterruptibly() + * reduced to a no-op. + */ + private static class TestInfiniteQueryableIndexSegment extends QueryableIndexSegment + { + + private static class InfiniteQueryableIndexStorageAdapter extends QueryableIndexStorageAdapter + { + CountDownLatch countDownLatch; + + public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch countDownLatch) + { + super(index); + this.countDownLatch = countDownLatch; + } + + @Override + public Sequence makeCursors( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + Granularity gran, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics) + .map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); + } + + private static class CursorNoAdvance implements Cursor + { + Cursor cursor; + CountDownLatch countDownLatch; + + public CursorNoAdvance(Cursor cursor, CountDownLatch countDownLatch) + { + this.cursor = cursor; + this.countDownLatch = countDownLatch; + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return cursor.getColumnSelectorFactory(); + } + + @Override + public DateTime getTime() + { + return cursor.getTime(); + } + + @Override + public void advance() + { + // Do nothing to simulate infinite rows + countDownLatch.countDown(); + } + + @Override + public void advanceUninterruptibly() + { + // Do nothing to simulate infinite rows + countDownLatch.countDown(); + + } + + @Override + public boolean isDone() + { + return false; + } + + @Override + public boolean isDoneOrInterrupted() + { + return cursor.isDoneOrInterrupted(); + } + + @Override + public void reset() + { + + } + } + } + + private final StorageAdapter testStorageAdaptor; + + public TestInfiniteQueryableIndexSegment(QueryableIndex index, SegmentId segmentId, CountDownLatch countDownLatch) + { + super(index, segmentId); + testStorageAdaptor = new InfiniteQueryableIndexStorageAdapter(index, countDownLatch); + } + + @Override + public StorageAdapter asStorageAdapter() + { + return testStorageAdaptor; + } + } + + + private static class ExceptionHandler implements Thread.UncaughtExceptionHandler + { + + Throwable exception; + + @Override + public void uncaughtException(Thread t, Throwable e) + { + exception = e; + } + + public Throwable getException() + { + return exception; + } + } + + @Test + public void testAdvanceWithInterruption() throws IOException, InterruptedException + { + + final int rowsBeforeInterrupt = 1000; + + CountDownLatch countDownLatch = new CountDownLatch(rowsBeforeInterrupt); + + infiniteFactSegment = new TestInfiniteQueryableIndexSegment( + JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), + SegmentId.dummy("facts"), + countDownLatch + ); + + countriesTable = JoinTestHelper.createCountriesIndexedTable(); + + Thread joinCursorThread = new Thread(() -> makeCursorAndAdvance()); + ExceptionHandler exceptionHandler = new ExceptionHandler(); + joinCursorThread.setUncaughtExceptionHandler(exceptionHandler); + joinCursorThread.start(); + + countDownLatch.await(1, TimeUnit.SECONDS); + joinCursorThread.interrupt(); + + // Wait for a max of 1 sec for the exception to be set. + for (int i = 0; i < 1000; i++) { + if (exceptionHandler.getException() == null) { + sleep(1); + } else { + assertTrue(exceptionHandler.getException() instanceof QueryInterruptedException); + return; + } + } + fail(); + } + + public void makeCursorAndAdvance() + { + + List joinableClauses = ImmutableList.of( + factToCountryOnIsoCode(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis( + null, + joinableClauses, + VirtualColumns.EMPTY + ); + + HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter = new HashJoinSegmentStorageAdapter( + infiniteFactSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + + Cursor cursor = Iterables.getOnlyElement(hashJoinSegmentStorageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ).toList()); + + ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + }); + + cursor.advance(); + } +} From 31730934159044e9444adec277132cf13054dd5c Mon Sep 17 00:00:00 2001 From: George Shiqi Wu Date: Mon, 30 Oct 2023 10:21:23 -0700 Subject: [PATCH 14/15] Handle status failures for streaming supervisors (#15174) * Cleanup logic * newline * remove whitespace * Fix log message * Add test class * PR changes --- .../firehose/ChatHandlerResource.java | 4 +- .../firehose/ChatHandlerResourceTest.java | 56 +++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java index 60a73153471c..1cd579d27340 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.initialization.jetty.BadRequestException; +import org.apache.druid.server.initialization.jetty.ServiceUnavailableException; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import javax.ws.rs.Path; @@ -73,6 +74,7 @@ public Object doTaskChat(@PathParam("id") String handlerId, @Context HttpHeaders return handler.get(); } - throw new BadRequestException(StringUtils.format("Can't find chatHandler for handler[%s]", handlerId)); + // Return HTTP 503 so SpecificTaskRetryPolicy retries in case the handler is not registered yet or has been registered before shutdown. + throw new ServiceUnavailableException(StringUtils.format("Can't find chatHandler for handler[%s]", handlerId)); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java new file mode 100644 index 000000000000..870636fb416a --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java @@ -0,0 +1,56 @@ +/* + * 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.segment.realtime.firehose; + +import com.google.common.base.Optional; +import org.apache.druid.server.initialization.jetty.ServiceUnavailableException; +import org.apache.druid.server.metrics.DataSourceTaskIdHolder; +import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +@RunWith(EasyMockRunner.class) +public class ChatHandlerResourceTest extends EasyMockSupport +{ + + @Mock + ChatHandlerProvider handlers; + @Mock + DataSourceTaskIdHolder dataSourceTaskIdHolder; + ChatHandlerResource chatHandlerResource; + + @Test + public void test_noHandlerFound() + { + String handlerId = "handlerId"; + EasyMock.expect(dataSourceTaskIdHolder.getTaskId()).andReturn(null); + EasyMock.expect(handlers.get(handlerId)).andReturn(Optional.absent()); + + replayAll(); + chatHandlerResource = new ChatHandlerResource(handlers, dataSourceTaskIdHolder); + Assert.assertThrows(ServiceUnavailableException.class, () -> chatHandlerResource.doTaskChat(handlerId, null)); + verifyAll(); + } +} From e6b7c36e740067742aa7f578e72e8a3c927572ca Mon Sep 17 00:00:00 2001 From: Suneet Saldanha Date: Mon, 30 Oct 2023 10:42:58 -0700 Subject: [PATCH 15/15] LoadRules with 0 replicas should be treated as handoff complete (#15274) * LoadRules with 0 replicas should be treated as handoff complete * fix it * pr feedback * fixit --- .../server/coordinator/rules/LoadRule.java | 37 +++++++++++++++++-- .../server/http/DataSourcesResource.java | 15 ++++---- .../coordinator/rules/LoadRuleTest.java | 9 ++++- .../coordinator/rules/PeriodLoadRuleTest.java | 1 + 4 files changed, 49 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index 5d7b724c8451..5463dd854986 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator.rules; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DruidServer; @@ -40,11 +41,14 @@ public abstract class LoadRule implements Rule */ private final boolean useDefaultTierForNull; + private final boolean shouldSegmentBeLoaded; + protected LoadRule(Map tieredReplicants, Boolean useDefaultTierForNull) { this.useDefaultTierForNull = Configs.valueOrDefault(useDefaultTierForNull, true); this.tieredReplicants = handleNullTieredReplicants(tieredReplicants, this.useDefaultTierForNull); validateTieredReplicants(this.tieredReplicants); + this.shouldSegmentBeLoaded = this.tieredReplicants.values().stream().reduce(0, Integer::sum) > 0; } @JsonProperty @@ -65,6 +69,18 @@ public void run(DataSegment segment, SegmentActionHandler handler) handler.replicateSegment(segment, getTieredReplicants()); } + + /** + * @return Whether a segment that matches this rule needs to be loaded on a tier. + * + * Used in making handoff decisions. + */ + @JsonIgnore + public boolean shouldMatchingSegmentBeLoaded() + { + return shouldSegmentBeLoaded; + } + /** * Returns the given {@code tieredReplicants} map unchanged if it is non-null (including empty). * Returns the following default values if the given map is null. @@ -73,10 +89,16 @@ public void run(DataSegment segment, SegmentActionHandler handler) *

  • If {@code useDefaultTierForNull} is false, returns an empty map. This causes segments to have a replication factor of 0 and not get assigned to any historical.
  • * */ - private static Map handleNullTieredReplicants(final Map tieredReplicants, boolean useDefaultTierForNull) + private static Map handleNullTieredReplicants( + final Map tieredReplicants, + boolean useDefaultTierForNull + ) { if (useDefaultTierForNull) { - return Configs.valueOrDefault(tieredReplicants, ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS)); + return Configs.valueOrDefault( + tieredReplicants, + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ); } else { return Configs.valueOrDefault(tieredReplicants, ImmutableMap.of()); } @@ -86,10 +108,17 @@ private static void validateTieredReplicants(final Map tieredRe { for (Map.Entry entry : tieredReplicants.entrySet()) { if (entry.getValue() == null) { - throw InvalidInput.exception("Invalid number of replicas for tier [%s]. Value must not be null.", entry.getKey()); + throw InvalidInput.exception( + "Invalid number of replicas for tier [%s]. Value must not be null.", + entry.getKey() + ); } if (entry.getValue() < 0) { - throw InvalidInput.exception("Invalid number of replicas for tier [%s]. Value [%d] must be positive.", entry.getKey(), entry.getValue()); + throw InvalidInput.exception( + "Invalid number of replicas for tier [%s]. Value [%d] must be positive.", + entry.getKey(), + entry.getValue() + ); } } } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 301d9631b7d8..e362745a72a1 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -96,6 +96,7 @@ import java.util.stream.Collectors; /** + * */ @Path("/druid/coordinator/v1/datasources") public class DataSourcesResource @@ -186,7 +187,8 @@ private interface MarkSegments @ResourceFilters(DatasourceResourceFilter.class) public Response markAsUsedAllNonOvershadowedSegments(@PathParam("dataSourceName") final String dataSourceName) { - MarkSegments markSegments = () -> segmentsMetadataManager.markAsUsedAllNonOvershadowedSegmentsInDataSource(dataSourceName); + MarkSegments markSegments = () -> segmentsMetadataManager.markAsUsedAllNonOvershadowedSegmentsInDataSource( + dataSourceName); return doMarkSegments("markAsUsedAllNonOvershadowedSegments", dataSourceName, markSegments); } @@ -480,7 +482,8 @@ public Response getDatasourceLoadstatus( return Response.ok( ImmutableMap.of( dataSourceName, - 100 * ((double) (segmentsLoadStatistics.getNumLoadedSegments()) / (double) segmentsLoadStatistics.getNumPublishedSegments()) + 100 * ((double) (segmentsLoadStatistics.getNumLoadedSegments()) + / (double) segmentsLoadStatistics.getNumPublishedSegments()) ) ).build(); } @@ -873,16 +876,14 @@ public Response isHandOffComplete( final DateTime now = DateTimes.nowUtc(); // A segment that is not eligible for load will never be handed off - boolean notEligibleForLoad = true; + boolean eligibleForLoad = false; for (Rule rule : rules) { if (rule.appliesTo(theInterval, now)) { - if (rule instanceof LoadRule) { - notEligibleForLoad = false; - } + eligibleForLoad = rule instanceof LoadRule && ((LoadRule) rule).shouldMatchingSegmentBeLoaded(); break; } } - if (notEligibleForLoad) { + if (!eligibleForLoad) { return Response.ok(true).build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 801df8ebd76d..1e43d89bdda4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -114,6 +114,7 @@ public void testLoadRuleAssignsSegments() final DataSegment segment = createDataSegment(DS_WIKI); LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2)); + Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); @@ -267,6 +268,7 @@ public void testSegmentsAreDroppedIfLoadRuleHasZeroReplicas() .build(); LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 0, Tier.T2, 0)); + Assert.assertFalse(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); @@ -284,7 +286,7 @@ public void testLoadIgnoresInvalidTiers() final DataSegment segment = createDataSegment(DS_WIKI); LoadRule rule = loadForever(ImmutableMap.of("invalidTier", 1, Tier.T1, 1)); - + Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "invalidTier", DS_WIKI)); @@ -347,6 +349,7 @@ public void testMaxLoadingQueueSize() .build(); final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); + Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats1 = runRuleAndGetStats(rule, dataSegment1, params); CoordinatorRunStats stats2 = runRuleAndGetStats(rule, dataSegment2, params); CoordinatorRunStats stats3 = runRuleAndGetStats(rule, dataSegment3, params); @@ -370,6 +373,7 @@ public void testSegmentIsAssignedOnlyToActiveServer() // Load rule requires 1 replica on each tier LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 1)); + Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); DataSegment segment = createDataSegment(DS_WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); @@ -427,7 +431,7 @@ public void testDropDuringDecommissioning() DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, segment1, segment2); final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 0)); - + Assert.assertFalse(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment1, params); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, segment1.getDataSource())); Assert.assertTrue(server1.getPeon().getSegmentsToDrop().contains(segment1)); @@ -531,6 +535,7 @@ public void testEquals() { EqualsVerifier.forClass(LoadRule.class) .withNonnullFields("tieredReplicants") + .withIgnoredFields("shouldSegmentBeLoaded") .usingGetClass() .verify(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java index 7b1dd2085f02..86ef92a3ed5c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java @@ -280,6 +280,7 @@ public void testEquals() { EqualsVerifier.forClass(PeriodLoadRule.class) .withNonnullFields("tieredReplicants") + .withIgnoredFields("shouldSegmentBeLoaded") .usingGetClass() .verify(); }