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 243f6626c664..8c41772aea69 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 @@ -283,10 +283,4 @@ public void updateStatus(Task task, TaskStatus status) { kubernetesTaskRunner.updateStatus(task, status); } - - @Override - public void updateLocation(Task task, TaskLocation location) - { - kubernetesTaskRunner.updateLocation(task, location); - } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 5c6c7c6b3ebe..ea3c2a19d1c2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -31,6 +31,9 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerUtils; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; @@ -47,6 +50,8 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -89,6 +94,8 @@ protected enum State private final KubernetesPeonClient kubernetesClient; private final ObjectMapper mapper; private final TaskStateListener stateListener; + private final List> listeners; + @MonotonicNonNull private LogWatch logWatch; @@ -99,7 +106,8 @@ protected KubernetesPeonLifecycle( KubernetesPeonClient kubernetesClient, TaskLogs taskLogs, ObjectMapper mapper, - TaskStateListener stateListener + TaskStateListener stateListener, + List> listeners ) { this.taskId = new K8sTaskId(task); @@ -108,6 +116,7 @@ protected KubernetesPeonLifecycle( this.taskLogs = taskLogs; this.mapper = mapper; this.stateListener = stateListener; + this.listeners = listeners; } /** @@ -178,7 +187,11 @@ protected synchronized TaskStatus join(long timeout) throws IllegalStateExceptio { try { updateState(new State[]{State.NOT_STARTED, State.PENDING}, State.RUNNING); - + TaskRunnerUtils.notifyLocationChanged( + listeners, + task.getId(), + getTaskLocation() + ); JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion( taskId, timeout, @@ -190,12 +203,14 @@ protected synchronized TaskStatus join(long timeout) throws IllegalStateExceptio finally { try { saveLogs(); + shutdown(); } catch (Exception e) { - log.warn(e, "Log processing failed for task [%s]", taskId); + log.warn(e, "Cleanup failed for task [%s]", taskId); + } + finally { + stopTask(); } - - stopTask(); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index bf4e3a712577..2998f3fc9212 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -21,9 +21,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; +import java.util.List; +import java.util.concurrent.Executor; + public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory { private final KubernetesPeonClient client; @@ -42,14 +47,15 @@ public KubernetesPeonLifecycleFactory( } @Override - public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener) + public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List> listeners) { return new KubernetesPeonLifecycle( task, client, taskLogs, mapper, - stateListener + stateListener, + listeners ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index a0a29dcbbb92..56de37c37980 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -146,16 +146,20 @@ public Optional streamTaskLog(String taskid, long offset) public ListenableFuture run(Task task) { synchronized (tasks) { - return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task)))) - .getResult(); + return tasks.computeIfAbsent(task.getId(), k -> { + ListenableFuture unused = exec.submit(() -> runTask(task)); + return new KubernetesWorkItem(task); + }).getResult(); } } protected ListenableFuture joinAsync(Task task) { synchronized (tasks) { - return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task)))) - .getResult(); + return tasks.computeIfAbsent(task.getId(), k -> { + ListenableFuture unused = exec.submit(() -> joinTask(task)); + return new KubernetesWorkItem(task); + }).getResult(); } } @@ -172,10 +176,12 @@ private TaskStatus joinTask(Task task) @VisibleForTesting protected TaskStatus doTask(Task task, boolean run) { + TaskStatus taskStatus = TaskStatus.failure(task.getId(), "Task execution never started"); try { KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build( task, - this::emitTaskStateMetrics + this::emitTaskStateMetrics, + listeners ); synchronized (tasks) { @@ -188,7 +194,6 @@ protected TaskStatus doTask(Task task, boolean run) workItem.setKubernetesPeonLifecycle(peonLifecycle); } - TaskStatus taskStatus; if (run) { taskStatus = peonLifecycle.run( adapter.fromTask(task), @@ -201,15 +206,17 @@ protected TaskStatus doTask(Task task, boolean run) config.getTaskTimeout().toStandardDuration().getMillis() ); } - - updateStatus(task, taskStatus); - return taskStatus; } catch (Exception e) { log.error(e, "Task [%s] execution caught an exception", task.getId()); + taskStatus = TaskStatus.failure(task.getId(), "Could not start task execution"); throw new RuntimeException(e); } + finally { + updateStatus(task, taskStatus); + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), TaskLocation.unknown()); + } } @VisibleForTesting @@ -242,13 +249,13 @@ protected void emitTaskStateMetrics(KubernetesPeonLifecycle.State state, String @Override public void updateStatus(Task task, TaskStatus status) { - TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); - } + KubernetesWorkItem workItem = tasks.get(task.getId()); + if (workItem != null && !workItem.getResult().isDone() && status.isComplete()) { + workItem.setResult(status); + } - @Override - public void updateLocation(Task task, TaskLocation location) - { - TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), location); + // Notify listeners even if the result is set to handle the shutdown case. + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); } @Override @@ -417,6 +424,16 @@ public void registerListener(TaskRunnerListener listener, Executor executor) final Pair listenerPair = Pair.of(listener, executor); log.debug("Registered listener [%s]", listener.getListenerId()); listeners.add(listenerPair); + + for (Map.Entry entry : tasks.entrySet()) { + if (entry.getValue().isRunning()) { + TaskRunnerUtils.notifyLocationChanged( + ImmutableList.of(listenerPair), + entry.getKey(), + entry.getValue().getLocation() + ); + } + } } @Override diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index 0d67c55b30aa..5fdcd9cfbb71 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -78,7 +78,7 @@ public class KubernetesTaskRunnerConfig @JsonProperty @NotNull // how long to wait for the jobs to be cleaned up. - private Period taskCleanupDelay = new Period("P2D"); + private Period taskCleanupDelay = new Period("PT1H"); @JsonProperty @NotNull diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesWorkItem.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesWorkItem.java index 94d4bbb67f63..b089b4dd2db0 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesWorkItem.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesWorkItem.java @@ -19,9 +19,10 @@ package org.apache.druid.k8s.overlord; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -36,9 +37,18 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem private final Task task; private KubernetesPeonLifecycle kubernetesPeonLifecycle = null; - public KubernetesWorkItem(Task task, ListenableFuture statusFuture) + private final SettableFuture result; + + public KubernetesWorkItem(Task task) + { + this(task, SettableFuture.create()); + } + + @VisibleForTesting + public KubernetesWorkItem(Task task, SettableFuture result) { - super(task.getId(), statusFuture); + super(task.getId(), result); + this.result = result; this.task = task; } @@ -51,7 +61,7 @@ protected synchronized void setKubernetesPeonLifecycle(KubernetesPeonLifecycle k protected synchronized void shutdown() { - if (this.kubernetesPeonLifecycle != null) { + if (this.kubernetesPeonLifecycle != null && !result.isDone()) { this.kubernetesPeonLifecycle.startWatchingLogs(); this.kubernetesPeonLifecycle.shutdown(); } @@ -119,4 +129,9 @@ public Task getTask() { return task; } + + public void setResult(TaskStatus status) + { + result.set(status); + } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java index 2a234ebc5786..2b180fb9dac0 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java @@ -20,8 +20,13 @@ package org.apache.druid.k8s.overlord; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Pair; + +import java.util.List; +import java.util.concurrent.Executor; public interface PeonLifecycleFactory { - KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener); + KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List> listeners); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java index a7a8156468f6..26fe5b98f0bd 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java @@ -32,12 +32,10 @@ public class JobResponse private static final EmittingLogger LOGGER = new EmittingLogger(JobResponse.class); private final Job job; - private final PeonPhase phase; - public JobResponse(@Nullable Job job, PeonPhase phase) + public JobResponse(@Nullable Job job) { this.job = job; - this.phase = phase; } public Job getJob() @@ -45,11 +43,6 @@ public Job getJob() return job; } - public PeonPhase getPhase() - { - return phase; - } - public long getJobDuration() { long duration = -1L; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 9fdc25fa6455..147ea732d0ca 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -101,13 +101,13 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time ); if (job == null) { log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); - return new JobResponse(null, PeonPhase.FAILED); + return new JobResponse(null); } if (job.getStatus().getSucceeded() != null) { - return new JobResponse(job, PeonPhase.SUCCEEDED); + return new JobResponse(job); } log.warn("Task %s failed with status %s", taskId, job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); + return new JobResponse(job); }); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java deleted file mode 100644 index 6efcd34872b8..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.common; - -import io.fabric8.kubernetes.api.model.Pod; - -import java.util.Arrays; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; - -public enum PeonPhase -{ - PENDING("Pending"), - SUCCEEDED("Succeeded"), - FAILED("Failed"), - UNKNOWN("Unknown"), - RUNNING("Running"); - - private static final Map PHASE_MAP = Arrays.stream(PeonPhase.values()) - .collect(Collectors.toMap( - PeonPhase::getPhase, - Function.identity() - )); - private final String phase; - - PeonPhase(String phase) - { - this.phase = phase; - } - - public String getPhase() - { - return phase; - } - - public static PeonPhase getPhaseFor(Pod pod) - { - if (pod == null) { - return UNKNOWN; - } - return PHASE_MAP.get(pod.getStatus().getPhase()); - } - -} 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 af5a6c39bb0b..80010b9e5396 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 @@ -332,13 +332,4 @@ public void test_updateStatus() runner.updateStatus(task, TaskStatus.running(ID)); verifyAll(); } - - @Test - public void test_updateLocation() - { - kubernetesTaskRunner.updateLocation(task, TaskLocation.unknown()); - replayAll(); - runner.updateLocation(task, TaskLocation.unknown()); - verifyAll(); - } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 1c6e429a3dc3..e984e449b282 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodBuilder; import io.fabric8.kubernetes.api.model.batch.v1.Job; @@ -31,11 +32,12 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; -import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -50,6 +52,8 @@ import java.io.IOException; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -65,6 +69,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport @Mock LogWatch logWatch; @Mock KubernetesPeonLifecycle.TaskStateListener stateListener; + List> listeners = ImmutableList.of(); + private ObjectMapper mapper; private Task task; private K8sTaskId k8sTaskId; @@ -86,7 +92,8 @@ public void test_run() throws IOException kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ) { @Override @@ -131,7 +138,8 @@ public void test_run_useTaskManager() throws IOException kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ) { @Override @@ -175,7 +183,8 @@ public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() throw kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ) { @Override @@ -224,7 +233,8 @@ public void test_run_whenExceptionRaised_setsRunnerTaskStateToNone() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ) { @Override @@ -268,15 +278,19 @@ public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(null, PeonPhase.FAILED)); + )).andReturn(new JobResponse(null)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()); + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); + EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent()); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); EasyMock.expectLastCall(); @@ -302,12 +316,15 @@ public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException @Test public void test_join() throws IOException { + Executor executor = EasyMock.mock(Executor.class); + TaskRunnerListener taskRunnerListener = EasyMock.mock(TaskRunnerListener.class); KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( task, kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + ImmutableList.of(Pair.of(taskRunnerListener, executor)) ); Job job = new JobBuilder() @@ -325,8 +342,12 @@ public void test_join() throws IOException EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + )).andReturn(new JobResponse(job)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.of( IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8) )); @@ -338,6 +359,10 @@ public void test_join() throws IOException EasyMock.expectLastCall().once(); logWatch.close(); EasyMock.expectLastCall(); + executor.execute(EasyMock.anyObject()); + EasyMock.expectLastCall(); + taskRunnerListener.locationChanged(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall(); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); @@ -359,7 +384,8 @@ public void test_join_whenCalledMultipleTimes_raisesIllegalStateException() thro kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); Job job = new JobBuilder() @@ -375,8 +401,15 @@ public void test_join_whenCalledMultipleTimes_raisesIllegalStateException() thro EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + )).andReturn(new JobResponse(job)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + + // Only update the location the first time, second call doesn't reach this point in the logic + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); + // Always try to delete the job + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true).times(2); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn( Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)) ); @@ -419,7 +452,8 @@ public void test_join_withoutTaskStatus_returnsFailedTaskStatus() throws IOExcep kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); Job job = new JobBuilder() @@ -435,8 +469,12 @@ public void test_join_withoutTaskStatus_returnsFailedTaskStatus() throws IOExcep EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + )).andReturn(new JobResponse(job)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent()); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); EasyMock.expectLastCall(); @@ -469,7 +507,8 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskStatus_returnsFaile kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); Job job = new JobBuilder() @@ -485,8 +524,12 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskStatus_returnsFaile EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + )).andReturn(new JobResponse(job)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andThrow(new IOException()); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); EasyMock.expectLastCall(); @@ -519,7 +562,8 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() th kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); Job job = new JobBuilder() @@ -535,8 +579,11 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() th EasyMock.eq(k8sTaskId), EasyMock.anyLong(), EasyMock.eq(TimeUnit.MILLISECONDS) - )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + )).andReturn(new JobResponse(job)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn( Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)) ); @@ -549,6 +596,9 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() th logWatch.close(); EasyMock.expectLastCall(); + // We should still try to cleanup the Job after + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); + Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); replayAll(); @@ -569,7 +619,8 @@ public void test_join_whenRuntimeExceptionThrownWhileWaitingForKubernetesJob_thr kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( @@ -578,6 +629,9 @@ public void test_join_whenRuntimeExceptionThrownWhileWaitingForKubernetesJob_thr EasyMock.eq(TimeUnit.MILLISECONDS) )).andThrow(new RuntimeException()); + EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn( + Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build()) + ); // We should still try to push logs EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); @@ -588,7 +642,7 @@ public void test_join_whenRuntimeExceptionThrownWhileWaitingForKubernetesJob_thr EasyMock.expectLastCall().once(); logWatch.close(); EasyMock.expectLastCall(); - + EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); replayAll(); @@ -608,7 +662,8 @@ public void test_shutdown_withNotStartedTaskState() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); peonLifecycle.shutdown(); } @@ -621,7 +676,8 @@ public void test_shutdown_withPendingTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -642,7 +698,8 @@ public void test_shutdown_withRunningTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -663,7 +720,8 @@ public void test_shutdown_withStoppedTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -678,7 +736,8 @@ public void test_streamLogs_withNotStartedTaskState() throws NoSuchFieldExceptio kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -693,7 +752,8 @@ public void test_streamLogs_withPendingTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -708,7 +768,8 @@ public void test_streamLogs_withRunningTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -731,7 +792,8 @@ public void test_streamLogs_withStoppedTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -747,7 +809,8 @@ public void test_getTaskLocation_withNotStartedTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -763,7 +826,8 @@ public void test_getTaskLocation_withPendingTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -779,7 +843,8 @@ public void test_getTaskLocation_withRunningTaskState_withoutPeonPod_returnsUnkn kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -801,7 +866,8 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithoutStatus_r kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -829,7 +895,8 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatus_retu kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -865,7 +932,8 @@ public void test_getTaskLocation_saveTaskLocation() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -901,7 +969,8 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatusWithT kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -938,7 +1007,8 @@ public void test_getTaskLocation_withStoppedTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener + stateListener, + listeners ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()).once(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java index 1f4a7281f649..579b7539d818 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java @@ -47,7 +47,7 @@ public void test_deserializable() throws IOException Assert.assertNull(config.getGraceTerminationPeriodSeconds()); Assert.assertTrue(config.isDisableClientProxy()); Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout()); - Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay()); + Assert.assertEquals(new Period("PT1H"), config.getTaskCleanupDelay()); Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval()); Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout()); Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors()); @@ -72,7 +72,7 @@ public void test_builder_preservesDefaults() Assert.assertNull(config.getGraceTerminationPeriodSeconds()); Assert.assertTrue(config.isDisableClientProxy()); Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout()); - Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay()); + Assert.assertEquals(new Period("PT1H"), config.getTaskCleanupDelay()); Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval()); Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout()); Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 36a7b4cfcd9c..e04ef6300362 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -28,8 +28,10 @@ import org.apache.commons.io.IOUtils; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; @@ -76,6 +78,9 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle; @Mock private ServiceEmitter emitter; + @Mock private Executor executor; + @Mock private TaskRunnerListener taskRunnerListener; + private KubernetesTaskRunnerConfig config; private KubernetesTaskRunner runner; private Task task; @@ -116,11 +121,7 @@ protected ListenableFuture joinAsync(Task task) { return tasks.computeIfAbsent( task.getId(), - k -> new KubernetesWorkItem( - task, - Futures.immediateFuture(TaskStatus.success(task.getId())) - ) - ).getResult(); + k -> new KubernetesWorkItem(task)).getResult(); } }; @@ -249,7 +250,7 @@ public void test_run_withExistingTask_returnsExistingWorkItem() } @Test - public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOException + public void test_run_whenExceptionThrown_throwsRuntimeException() throws Exception { Job job = new JobBuilder() .withNewMetadata() @@ -269,11 +270,89 @@ public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOExcep replayAll(); ListenableFuture future = runner.run(task); + TaskStatus taskStatus = future.get(); + Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); + Assert.assertEquals("Could not start task execution", taskStatus.getErrorMsg()); + verifyAll(); + } - Exception e = Assert.assertThrows(ExecutionException.class, future::get); - Assert.assertTrue(e.getCause() instanceof RuntimeException); + @Test + public void test_run_updateStatus() throws ExecutionException, InterruptedException + { + KubernetesTaskRunner runner = new KubernetesTaskRunner( + taskAdapter, + config, + peonClient, + httpClient, + new TestPeonLifecycleFactory(kubernetesPeonLifecycle), + emitter + ); + KubernetesWorkItem workItem = new KubernetesWorkItem(task); + runner.tasks.put(task.getId(), workItem); + TaskStatus completeTaskStatus = TaskStatus.success(task.getId()); + + replayAll(); + runner.updateStatus(task, completeTaskStatus); verifyAll(); + + assertTrue(workItem.getResult().isDone()); + assertEquals(completeTaskStatus, workItem.getResult().get()); + } + + @Test + public void test_run_updateStatus_running() + { + KubernetesTaskRunner runner = new KubernetesTaskRunner( + taskAdapter, + config, + peonClient, + httpClient, + new TestPeonLifecycleFactory(kubernetesPeonLifecycle), + emitter + ); + KubernetesWorkItem workItem = new KubernetesWorkItem(task); + runner.tasks.put(task.getId(), workItem); + TaskStatus runningTaskStatus = TaskStatus.running(task.getId()); + + replayAll(); + runner.updateStatus(task, runningTaskStatus); + verifyAll(); + + assertFalse(workItem.getResult().isDone()); + } + + @Test + public void test_registerListener_runningTask() + { + KubernetesTaskRunner runner = new KubernetesTaskRunner( + taskAdapter, + config, + peonClient, + httpClient, + new TestPeonLifecycleFactory(kubernetesPeonLifecycle), + emitter + ); + + KubernetesPeonLifecycle runningKubernetesPeonLifecycle = EasyMock.mock(KubernetesPeonLifecycle.class); + EasyMock.expect(runningKubernetesPeonLifecycle.getState()).andReturn(KubernetesPeonLifecycle.State.RUNNING); + EasyMock.expect(runningKubernetesPeonLifecycle.getTaskLocation()).andReturn(TaskLocation.unknown()); + KubernetesWorkItem workItem = new KubernetesWorkItem(task); + workItem.setKubernetesPeonLifecycle(runningKubernetesPeonLifecycle); + runner.tasks.put(task.getId(), workItem); + + Executor executor = EasyMock.mock(Executor.class); + TaskRunnerListener taskRunnerListener = EasyMock.mock(TaskRunnerListener.class); + executor.execute(EasyMock.anyObject()); + EasyMock.expectLastCall(); + taskRunnerListener.locationChanged(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall(); + + replayAll(); + EasyMock.replay(runningKubernetesPeonLifecycle); + runner.registerListener(taskRunnerListener, executor); + verifyAll(); + EasyMock.verify(runningKubernetesPeonLifecycle); } @Test @@ -303,16 +382,15 @@ public void test_join_withExistingTask_returnsExistingWorkItem() } @Test - public void test_join_whenExceptionThrown_throwsRuntimeException() + public void test_join_whenExceptionThrown_throwsRuntimeException() throws ExecutionException, InterruptedException { EasyMock.expect(kubernetesPeonLifecycle.join(EasyMock.anyLong())).andThrow(new IllegalStateException()); replayAll(); ListenableFuture future = runner.joinAsync(task); - - Exception e = Assert.assertThrows(ExecutionException.class, future::get); - Assert.assertTrue(e.getCause() instanceof RuntimeException); + TaskStatus taskStatus = future.get(); + Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); verifyAll(); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java index 7d17193b1714..f2f398658e05 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java @@ -56,6 +56,7 @@ public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException() null, null, null, + null, null )); @@ -66,6 +67,7 @@ public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException() null, null, null, + null, null )) ); @@ -80,6 +82,8 @@ public void test_shutdown_withoutKubernetesPeonLifecycle() @Test public void test_shutdown_withKubernetesPeonLifecycle() { + KubernetesWorkItem workItem = new KubernetesWorkItem(task); + kubernetesPeonLifecycle.shutdown(); EasyMock.expectLastCall(); kubernetesPeonLifecycle.startWatchingLogs(); @@ -87,7 +91,6 @@ public void test_shutdown_withKubernetesPeonLifecycle() replayAll(); workItem.setKubernetesPeonLifecycle(kubernetesPeonLifecycle); - workItem.shutdown(); verifyAll(); } @@ -158,6 +161,7 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_returnsPending() null, null, null, + null, null )); @@ -172,6 +176,7 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inPendingState_r null, null, null, + null, null ) { @Override @@ -194,6 +199,7 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inRunningState_r null, null, null, + null, null ) { @Override @@ -216,6 +222,7 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inStoppedState_r null, null, null, + null, null ) { @Override @@ -244,6 +251,7 @@ public void test_streamTaskLogs_withKubernetesPeonLifecycle() null, null, null, + null, null )); Assert.assertFalse(workItem.streamTaskLogs().isPresent()); @@ -263,6 +271,7 @@ public void test_getLocation_withKubernetesPeonLifecycle() null, null, null, + null, null )); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java index 8b8c43c0d71c..fa0f79bc1d20 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java @@ -20,6 +20,11 @@ package org.apache.druid.k8s.overlord; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Pair; + +import java.util.List; +import java.util.concurrent.Executor; public class TestPeonLifecycleFactory implements PeonLifecycleFactory { @@ -31,7 +36,7 @@ public TestPeonLifecycleFactory(KubernetesPeonLifecycle kubernetesPeonLifecycle) } @Override - public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener) + public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List> listeners) { return kubernetesPeonLifecycle; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java index 2e2043578aa1..cf9f345fd7a5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java @@ -39,7 +39,7 @@ void testCompletionTime() .endStatus() .build(); - JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED); + JobResponse response = new JobResponse(job); Assertions.assertEquals(58000L, response.getJobDuration()); } @@ -56,7 +56,7 @@ void testNoDuration() .endStatus() .build(); - JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED); + JobResponse response = new JobResponse(job); Assertions.assertEquals(-1, response.getJobDuration()); } @@ -70,7 +70,7 @@ void testMakingCodeCoverageHappy() .endMetadata() .build(); - JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED); + JobResponse response = new JobResponse(job); Assertions.assertEquals(-1, response.getJobDuration()); } @@ -78,7 +78,7 @@ void testMakingCodeCoverageHappy() @Test void testNullJob() { - JobResponse response = new JobResponse(null, PeonPhase.SUCCEEDED); + JobResponse response = new JobResponse(null); long duration = response.getJobDuration(); Assertions.assertEquals(-1, duration); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index f6096b675d6c..cde7faa473bc 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -153,7 +153,6 @@ void test_waitForPeonJobCompletion_withSuccessfulJob_returnsJobResponseWithJobAn TimeUnit.SECONDS ); - Assertions.assertEquals(PeonPhase.SUCCEEDED, jobResponse.getPhase()); Assertions.assertNotNull(jobResponse.getJob()); } @@ -178,7 +177,6 @@ void test_waitForPeonJobCompletion_withFailedJob_returnsJobResponseWithJobAndFai TimeUnit.SECONDS ); - Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase()); Assertions.assertNotNull(jobResponse.getJob()); } @@ -191,7 +189,6 @@ void test_waitforPeonJobCompletion_withoutRunningJob_returnsJobResponseWithEmpty TimeUnit.SECONDS ); - Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase()); Assertions.assertNull(jobResponse.getJob()); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 098161685883..2cc5bf15c65a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -38,13 +38,11 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; -import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesClientApi; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonCommandContext; -import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.junit.jupiter.api.BeforeEach; @@ -184,9 +182,8 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception assertEquals(task, taskFromPod); - JobResponse jobStatusResult = peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES); + peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES); thread.join(); - assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase()); // as long as there were no exceptions we are good! assertEquals(expectedLogs, actualLogs); // cleanup my job diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedEphemeralOutput.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedEphemeralOutput.yaml index 30960cdbc668..741a032eb6c9 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedEphemeralOutput.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedEphemeralOutput.yaml @@ -62,4 +62,4 @@ spec: ephemeral-storage: 1Gi hostname: "id-3e70afe5cd823dfc7dd308eea616426b" restartPolicy: "Never" - ttlSecondsAfterFinished: 172800 \ No newline at end of file + ttlSecondsAfterFinished: 3600 \ No newline at end of file diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml index 70b8b7c1d242..73f31ddfb508 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml @@ -105,4 +105,4 @@ spec: name: "graveyard" - emptyDir: {} name: "kubexit" - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutputOrder.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutputOrder.yaml index 70b8b7c1d242..73f31ddfb508 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutputOrder.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutputOrder.yaml @@ -105,4 +105,4 @@ spec: name: "graveyard" - emptyDir: {} name: "kubexit" - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index 2cef837f3972..004fed9585af 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -16,7 +16,7 @@ metadata: spec: activeDeadlineSeconds: 14400 backoffLimit: 0 - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 template: metadata: labels: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index cf16c49c5db1..b6ca8a2cefe6 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -16,7 +16,7 @@ metadata: spec: activeDeadlineSeconds: 14400 backoffLimit: 0 - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 template: metadata: labels: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index d72d0ef37b03..8ecdaf50b012 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -16,7 +16,7 @@ metadata: spec: activeDeadlineSeconds: 14400 backoffLimit: 0 - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 template: metadata: labels: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml index a230ac913a60..547887e90847 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml @@ -16,7 +16,7 @@ metadata: spec: activeDeadlineSeconds: 14400 backoffLimit: 0 - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 template: metadata: labels: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml index e46de1337883..ecd9416c563a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml @@ -104,4 +104,4 @@ spec: name: "graveyard" - emptyDir: {} name: "kubexit" - ttlSecondsAfterFinished: 172800 + ttlSecondsAfterFinished: 3600 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleContainerOutput.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleContainerOutput.yaml index f270368fb552..7afc393c56af 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleContainerOutput.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleContainerOutput.yaml @@ -57,4 +57,4 @@ spec: cpu: "1000m" memory: "2400000000" restartPolicy: "Never" - ttlSecondsAfterFinished: 172800 \ No newline at end of file + ttlSecondsAfterFinished: 3600 \ No newline at end of file diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index 636897c610c9..70a84d80e49a 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -40,6 +40,7 @@ import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -370,7 +371,7 @@ Pair, Map>> checkSegment // Pair max(created_date), interval -> list> Pair, Map>> baseSegmentsSnapshot = getMaxCreateDateAndBaseSegments( - metadataStorageCoordinator.retrieveUsedSegmentsAndCreatedDates(spec.getBaseDataSource()) + metadataStorageCoordinator.retrieveUsedSegmentsAndCreatedDates(spec.getBaseDataSource(), Intervals.ETERNITY) ); // baseSegments are used to create HadoopIndexTask Map> baseSegments = baseSegmentsSnapshot.rhs; 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 c108c7d679e9..f2260b055a96 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 @@ -47,6 +47,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.BrokerClient; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.key.ClusterBy; @@ -69,7 +70,10 @@ import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; +import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.batch.parallel.TombstoneHelper; import org.apache.druid.indexing.overlord.SegmentPublishResult; @@ -962,7 +966,11 @@ private List generateSegmentIdsWithShardSpecs( ); } else { final RowKeyReader keyReader = clusterBy.keyReader(signature); - return generateSegmentIdsWithShardSpecsForAppend(destination, partitionBoundaries, keyReader); + return generateSegmentIdsWithShardSpecsForAppend( + destination, + partitionBoundaries, + keyReader, + MultiStageQueryContext.validateAndGetTaskLockType(QueryContext.of(task.getQuerySpec().getQuery().getContext()), false)); } } @@ -972,7 +980,8 @@ private List generateSegmentIdsWithShardSpecs( private List generateSegmentIdsWithShardSpecsForAppend( final DataSourceMSQDestination destination, final ClusterByPartitions partitionBoundaries, - final RowKeyReader keyReader + final RowKeyReader keyReader, + final TaskLockType taskLockType ) throws IOException { final Granularity segmentGranularity = destination.getSegmentGranularity(); @@ -998,7 +1007,7 @@ private List generateSegmentIdsWithShardSpecsForAppend( false, NumberedPartialShardSpec.instance(), LockGranularity.TIME_CHUNK, - TaskLockType.SHARED + taskLockType ) ); } @@ -1399,6 +1408,10 @@ private void publishAllSegments(final Set segments) throws IOExcept (DataSourceMSQDestination) task.getQuerySpec().getDestination(); final Set segmentsWithTombstones = new HashSet<>(segments); int numTombstones = 0; + final TaskLockType taskLockType = MultiStageQueryContext.validateAndGetTaskLockType( + QueryContext.of(task.getQuerySpec().getQuery().getContext()), + destination.isReplaceTimeChunks() + ); if (destination.isReplaceTimeChunks()) { final List intervalsToDrop = findIntervalsToDrop(Preconditions.checkNotNull(segments, "segments")); @@ -1441,7 +1454,7 @@ private void publishAllSegments(final Set segments) throws IOExcept } performSegmentPublish( context.taskActionClient(), - SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones) + createOverwriteAction(taskLockType, segmentsWithTombstones) ); } } else if (!segments.isEmpty()) { @@ -1458,7 +1471,7 @@ private void publishAllSegments(final Set segments) throws IOExcept // Append mode. performSegmentPublish( context.taskActionClient(), - SegmentTransactionalInsertAction.appendAction(segments, null, null) + createAppendAction(segments, taskLockType) ); } @@ -1467,6 +1480,34 @@ private void publishAllSegments(final Set segments) throws IOExcept task.emitMetric(context.emitter(), "ingest/segments/count", segmentsWithTombstones.size()); } + private static TaskAction createAppendAction( + Set segments, + TaskLockType taskLockType + ) + { + if (taskLockType.equals(TaskLockType.APPEND)) { + return SegmentTransactionalAppendAction.forSegments(segments); + } else if (taskLockType.equals(TaskLockType.SHARED)) { + return SegmentTransactionalInsertAction.appendAction(segments, null, null); + } else { + throw DruidException.defensive("Invalid lock type [%s] received for append action", taskLockType); + } + } + + private TaskAction createOverwriteAction( + TaskLockType taskLockType, + Set segmentsWithTombstones + ) + { + if (taskLockType.equals(TaskLockType.REPLACE)) { + return SegmentTransactionalReplaceAction.create(segmentsWithTombstones); + } else if (taskLockType.equals(TaskLockType.EXCLUSIVE)) { + return SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones); + } else { + throw DruidException.defensive("Invalid lock type [%s] received for overwrite action", taskLockType); + } + } + /** * When doing an ingestion with {@link DataSourceMSQDestination#isReplaceTimeChunks()}, finds intervals * containing data that should be dropped. @@ -2282,7 +2323,7 @@ private static Map copyOfStageRuntimesEndingAtCurrentTime( */ static void performSegmentPublish( final TaskActionClient client, - final SegmentTransactionalInsertAction action + final TaskAction action ) throws IOException { try { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 43967e7d748a..3cdf706ba163 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -51,6 +51,8 @@ import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; @@ -204,12 +206,19 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception { // If we're in replace mode, acquire locks for all intervals before declaring the task ready. if (isIngestion(querySpec) && ((DataSourceMSQDestination) querySpec.getDestination()).isReplaceTimeChunks()) { + final TaskLockType taskLockType = + MultiStageQueryContext.validateAndGetTaskLockType(QueryContext.of(querySpec.getQuery().getContext()), true); final List intervals = ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks(); - log.debug("Task[%s] trying to acquire[%s] locks for intervals[%s] to become ready", getId(), TaskLockType.EXCLUSIVE, intervals); + log.debug( + "Task[%s] trying to acquire[%s] locks for intervals[%s] to become ready", + getId(), + taskLockType, + intervals + ); for (final Interval interval : intervals) { final TaskLock taskLock = - taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)); + taskActionClient.submit(new TimeChunkLockTryAcquireAction(taskLockType, interval)); if (taskLock == null) { return false; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java index f53c2cf7c540..93f24cbdff6d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java @@ -45,6 +45,7 @@ public class ExternalSegment extends RowBasedSegment { private final InputSource inputSource; + private final RowSignature signature; public static final String SEGMENT_ID = "__external"; /** @@ -145,6 +146,7 @@ public void cleanup(CloseableIterator iterFromMake) signature ); this.inputSource = inputSource; + this.signature = signature; } /** @@ -154,4 +156,12 @@ public InputSource externalInputSource() { return inputSource; } + + /** + * Returns the signature of the external input source + */ + public RowSignature signature() + { + return signature; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java index fbe82c240dbf..fc9f59ad32c6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java @@ -21,6 +21,8 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; @@ -32,6 +34,7 @@ import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.IndexedInts; import javax.annotation.Nullable; @@ -48,16 +51,19 @@ public class ExternalColumnSelectorFactory implements ColumnSelectorFactory private final ColumnSelectorFactory delegate; private final InputSource inputSource; + private final RowSignature rowSignature; private final SimpleSettableOffset offset; public ExternalColumnSelectorFactory( final ColumnSelectorFactory delgate, final InputSource inputSource, + final RowSignature rowSignature, final SimpleSettableOffset offset ) { this.delegate = delgate; this.inputSource = inputSource; + this.rowSignature = rowSignature; this.offset = offset; } @@ -67,6 +73,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) return new DimensionSelector() { final DimensionSelector delegateDimensionSelector = delegate.makeDimensionSelector(dimensionSpec); + final ExpressionType expressionType = ExpressionType.fromColumnType(dimensionSpec.getOutputType()); @Override public IndexedInts getRow() @@ -97,7 +104,10 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) public Object getObject() { try { - return delegateDimensionSelector.getObject(); + if (expressionType == null) { + return delegateDimensionSelector.getObject(); + } + return ExprEval.ofType(expressionType, delegateDimensionSelector.getObject()).value(); } catch (Exception e) { throw createException(e, dimensionSpec.getDimension(), inputSource, offset); @@ -144,6 +154,9 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) return new ColumnValueSelector() { final ColumnValueSelector delegateColumnValueSelector = delegate.makeColumnValueSelector(columnName); + final ExpressionType expressionType = ExpressionType.fromColumnType( + rowSignature.getColumnType(columnName).orElse(null) + ); @Override public double getDouble() @@ -195,7 +208,10 @@ public boolean isNull() public Object getObject() { try { - return delegateColumnValueSelector.getObject(); + if (expressionType == null) { + return delegateColumnValueSelector.getObject(); + } + return ExprEval.ofType(expressionType, delegateColumnValueSelector.getObject()).value(); } catch (Exception e) { throw createException(e, columnName, inputSource, offset); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 1541d314f215..278a9c251dea 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -392,6 +392,7 @@ private ColumnSelectorFactory wrapColumnSelectorFactoryIfNeeded(final ColumnSele return new ExternalColumnSelectorFactory( baseColumnSelectorFactory, ((ExternalSegment) segment).externalInputSource(), + ((ExternalSegment) segment).signature(), cursorOffset ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index de48387db200..d38fa1a8dc64 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -226,12 +226,15 @@ public QueryResponse runQuery(final DruidQuery druidQuery) fieldMapping.stream().map(f -> f.right).collect(Collectors.toList()) ); - destination = new DataSourceMSQDestination( + final DataSourceMSQDestination dataSourceMSQDestination = new DataSourceMSQDestination( targetDataSource, segmentGranularityObject, segmentSortOrder, replaceTimeChunks ); + MultiStageQueryContext.validateAndGetTaskLockType(sqlQueryContext, + dataSourceMSQDestination.isReplaceTimeChunks()); + destination = dataSourceMSQDestination; } else { final MSQSelectDestination msqSelectDestination = MultiStageQueryContext.getSelectDestination(sqlQueryContext); if (msqSelectDestination.equals(MSQSelectDestination.TASKREPORT)) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 77b11a287687..f6caa6da0590 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -25,6 +25,10 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.exec.SegmentSource; @@ -78,6 +82,11 @@ * ingested via MSQ. If set to 'none', arrays are not allowed to be ingested in MSQ. If set to 'array', array types * can be ingested as expected. If set to 'mvd', numeric arrays can not be ingested, and string arrays will be * ingested as MVDs (this is kept for legacy purpose). + * + *
  • taskLockType: Temporary flag to allow MSQ to use experimental lock types. Valid values are present in + * {@link TaskLockType}. If the flag is not set, msq uses {@link TaskLockType#EXCLUSIVE} for replace queries and + * {@link TaskLockType#SHARED} for insert queries. + * * **/ public class MultiStageQueryContext @@ -350,4 +359,53 @@ static IndexSpec decodeIndexSpec(@Nullable final Object indexSpecObject, final O throw QueryContexts.badValueException(CTX_INDEX_SPEC, "an indexSpec", indexSpecObject); } } + + /** + * This method is used to validate and get the taskLockType from the queryContext. + * If the queryContext does not contain the taskLockType, then {@link TaskLockType#EXCLUSIVE} is used for replace queries and + * {@link TaskLockType#SHARED} is used for insert queries. + * If the queryContext contains the taskLockType, then it is validated and returned. + */ + public static TaskLockType validateAndGetTaskLockType(QueryContext queryContext, boolean isReplaceQuery) + { + final TaskLockType taskLockType = QueryContexts.getAsEnum( + Tasks.TASK_LOCK_TYPE, + queryContext.getString(Tasks.TASK_LOCK_TYPE, null), + TaskLockType.class + ); + if (taskLockType == null) { + if (isReplaceQuery) { + return TaskLockType.EXCLUSIVE; + } else { + return TaskLockType.SHARED; + } + } + final String appendErrorMessage = StringUtils.format( + " Please use [%s] key in the context parameter and use one of the TaskLock types as mentioned earlier or " + + "remove this key for automatic lock type selection", Tasks.TASK_LOCK_TYPE); + + if (isReplaceQuery && !(taskLockType.equals(TaskLockType.EXCLUSIVE) || taskLockType.equals(TaskLockType.REPLACE))) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "TaskLock must be of type [%s] or [%s] for a REPLACE query. Found invalid type [%s] set." + + appendErrorMessage, + TaskLockType.EXCLUSIVE, + TaskLockType.REPLACE, + taskLockType + ); + } + if (!isReplaceQuery && !(taskLockType.equals(TaskLockType.SHARED) || taskLockType.equals(TaskLockType.APPEND))) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "TaskLock must be of type [%s] or [%s] for an INSERT query. Found invalid type [%s] set." + + appendErrorMessage, + TaskLockType.SHARED, + TaskLockType.APPEND, + taskLockType + ); + } + return taskLockType; + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 2b152cfbe1c4..282dacc115c2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.java.util.common.ISE; @@ -857,6 +858,100 @@ public void testScanWithOrderByOnDoubleArray() .verifyResults(); } + @Test + public void testScanExternBooleanArray() + { + final List expectedRows = Collections.singletonList( + new Object[]{Arrays.asList(1L, 0L, null)} + ); + + RowSignature scanSignature = RowSignature.builder() + .add("a_bool", ColumnType.LONG_ARRAY) + .build(); + + Query expectedQuery = newScanQueryBuilder() + .dataSource( + new ExternalDataSource( + new InlineInputSource("{\"a_bool\":[true,false,null]}"), + new JsonInputFormat(null, null, null, null, null), + scanSignature + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a_bool") + .context(defaultScanQueryContext(context, scanSignature)) + .build(); + + testSelectQuery().setSql("SELECT a_bool FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\": \"inline\", \"data\":\"{\\\"a_bool\\\":[true,false,null]}\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"a_bool\", \"type\": \"ARRAY\"}]'\n" + + " )\n" + + ")") + .setQueryContext(context) + .setExpectedMSQSpec(MSQSpec + .builder() + .query(expectedQuery) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("a_bool", "a_bool") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setExpectedRowSignature(scanSignature) + .setExpectedResultRows(expectedRows) + .verifyResults(); + } + + @Test + public void testScanExternArrayWithNonConvertibleType() + { + final List expectedRows = Collections.singletonList( + new Object[]{Arrays.asList(null, null)} + ); + + RowSignature scanSignature = RowSignature.builder() + .add("a_bool", ColumnType.LONG_ARRAY) + .build(); + + Query expectedQuery = newScanQueryBuilder() + .dataSource( + new ExternalDataSource( + new InlineInputSource("{\"a_bool\":[\"Test\",\"Test2\"]}"), + new JsonInputFormat(null, null, null, null, null), + scanSignature + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a_bool") + .context(defaultScanQueryContext(context, scanSignature)) + .build(); + + testSelectQuery().setSql("SELECT a_bool FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\": \"inline\", \"data\":\"{\\\"a_bool\\\":[\\\"Test\\\",\\\"Test2\\\"]}\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"a_bool\", \"type\": \"ARRAY\"}]'\n" + + " )\n" + + ")") + .setQueryContext(context) + .setExpectedMSQSpec(MSQSpec + .builder() + .query(expectedQuery) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("a_bool", "a_bool") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setExpectedRowSignature(scanSignature) + .setExpectedResultRows(expectedRows) + .verifyResults(); + } + private List expectedMultiValueFooRowsToArray() { List expectedRows = new ArrayList<>(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index 4b77dd78b339..612dee3bbd83 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -22,7 +22,9 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; @@ -45,6 +47,7 @@ import java.io.File; import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -175,10 +178,10 @@ public void testInsertTimeNullFault() .build(); final String sql = "INSERT INTO foo1\n" - + "SELECT TIME_PARSE(dim1) AS __time, dim1 as cnt\n" - + "FROM foo\n" - + "PARTITIONED BY DAY\n" - + "CLUSTERED BY dim1"; + + "SELECT TIME_PARSE(dim1) AS __time, dim1 as cnt\n" + + "FROM foo\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY dim1"; testIngestQuery() .setSql(sql) @@ -349,8 +352,9 @@ public void testUnionAllWithDifferentColumnNames() DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general" - ).expectMessageContains("SQL requires union between two tables and column names queried for each table are different " - + "Left: [dim2, dim1, m1], Right: [dim1, dim2, m1].")) + ).expectMessageContains( + "SQL requires union between two tables and column names queried for each table are different " + + "Left: [dim2, dim1, m1], Right: [dim1, dim2, m1].")) .verifyPlanningErrors(); } @@ -374,4 +378,47 @@ public void testTopLevelUnionAllWithJoins() "SQL requires union between inputs that are not simple table scans and involve a filter or aliasing")) .verifyPlanningErrors(); } + + @Test + public void testInsertWithReplaceAndExcludeLocks() + { + for (TaskLockType taskLockType : new TaskLockType[]{TaskLockType.EXCLUSIVE, TaskLockType.REPLACE}) { + testLockTypes( + taskLockType, + "INSERT INTO foo1 select * from foo partitioned by day", + "TaskLock must be of type [SHARED] or [APPEND] for an INSERT query" + ); + } + } + + @Test + public void testReplaceWithAppendAndSharedLocks() + { + for (TaskLockType taskLockType : new TaskLockType[]{TaskLockType.APPEND, TaskLockType.SHARED}) { + testLockTypes( + taskLockType, + "REPLACE INTO foo1 overwrite ALL select * from foo partitioned by day", + "TaskLock must be of type [EXCLUSIVE] or [REPLACE] for a REPLACE query" + ); + } + } + + private void testLockTypes(TaskLockType contextTaskLockType, String sql, String errorMessage) + { + Map context = new HashMap<>(DEFAULT_MSQ_CONTEXT); + context.put(Tasks.TASK_LOCK_TYPE, contextTaskLockType.name()); + testIngestQuery() + .setSql( + sql + ) + .setQueryContext(context) + .setExpectedValidationErrorMatcher( + new DruidExceptionMatcher( + DruidException.Persona.USER, + DruidException.Category.INVALID_INPUT, + "general" + ).expectMessageContains( + errorMessage)) + .verifyPlanningErrors(); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index b43dd72e88c8..2314c10d7e6e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -28,6 +28,8 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -59,6 +61,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -67,6 +70,16 @@ @RunWith(Parameterized.class) public class MSQInsertTest extends MSQTestBase { + + private static final String WITH_APPEND_LOCK = "WITH_APPEND_LOCK"; + private static final Map QUERY_CONTEXT_WITH_APPEND_LOCK = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put( + Tasks.TASK_LOCK_TYPE, + TaskLockType.APPEND.name().toLowerCase(Locale.ENGLISH) + ) + .build(); private final HashFunction fn = Hashing.murmur3_128(); @Parameterized.Parameters(name = "{index}:with context {0}") @@ -76,7 +89,8 @@ public static Collection data() {DEFAULT, DEFAULT_MSQ_CONTEXT}, {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, - {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT} + {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT}, + {WITH_APPEND_LOCK, QUERY_CONTEXT_WITH_APPEND_LOCK} }; return Arrays.asList(data); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 0a43fdaea721..144e74b084e9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -20,10 +20,14 @@ package org.apache.druid.msq.exec; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestFileUtils; @@ -54,6 +58,17 @@ @RunWith(Parameterized.class) public class MSQReplaceTest extends MSQTestBase { + + private static final String WITH_REPLACE_LOCK = "WITH_REPLACE_LOCK"; + private static final Map QUERY_CONTEXT_WITH_REPLACE_LOCK = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put( + Tasks.TASK_LOCK_TYPE, + StringUtils.toLowerCase(TaskLockType.REPLACE.name()) + ) + .build(); + @Parameterized.Parameters(name = "{index}:with context {0}") public static Collection data() { @@ -61,7 +76,8 @@ public static Collection data() {DEFAULT, DEFAULT_MSQ_CONTEXT}, {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, - {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT} + {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT}, + {WITH_REPLACE_LOCK, QUERY_CONTEXT_WITH_REPLACE_LOCK} }; return Arrays.asList(data); } 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 219af6a31883..b63ee479e202 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 @@ -97,7 +97,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; @RunWith(Parameterized.class) @@ -114,7 +113,7 @@ public class MSQSelectTest extends MSQTestBase .put(MultiStageQueryContext.CTX_ROWS_PER_PAGE, 2) .put( MultiStageQueryContext.CTX_SELECT_DESTINATION, - MSQSelectDestination.DURABLESTORAGE.getName().toLowerCase(Locale.ENGLISH) + StringUtils.toLowerCase(MSQSelectDestination.DURABLESTORAGE.getName()) ) .build(); @@ -124,7 +123,7 @@ public class MSQSelectTest extends MSQTestBase .putAll(DEFAULT_MSQ_CONTEXT) .put( MultiStageQueryContext.CTX_SELECT_DESTINATION, - MSQSelectDestination.DURABLESTORAGE.getName().toLowerCase(Locale.ENGLISH) + StringUtils.toLowerCase(MSQSelectDestination.DURABLESTORAGE.getName()) ) .build(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java index 897e57c93bc8..31b3272b74ff 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java @@ -28,7 +28,9 @@ import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.overlord.SegmentPublishResult; @@ -121,10 +123,17 @@ public RetType submit(TaskAction taskAction) ).collect(Collectors.toSet()); } } else if (taskAction instanceof SegmentTransactionalInsertAction) { - // Always OK. final Set segments = ((SegmentTransactionalInsertAction) taskAction).getSegments(); publishedSegments.addAll(segments); return (RetType) SegmentPublishResult.ok(segments); + } else if (taskAction instanceof SegmentTransactionalReplaceAction) { + final Set segments = ((SegmentTransactionalReplaceAction) taskAction).getSegments(); + publishedSegments.addAll(segments); + return (RetType) SegmentPublishResult.ok(segments); + } else if (taskAction instanceof SegmentTransactionalAppendAction) { + final Set segments = ((SegmentTransactionalAppendAction) taskAction).getSegments(); + publishedSegments.addAll(segments); + return (RetType) SegmentPublishResult.ok(segments); } else { return null; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsToReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsToReplaceAction.java new file mode 100644 index 000000000000..78e6ada5c1e2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsToReplaceAction.java @@ -0,0 +1,209 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.batch.parallel.AbstractBatchSubtask; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.Interval; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This action exists in addition to retrieveUsedSegmentsAction because that action suffers + * from a race condition described by the following sequence of events: + * + * -Segments S1, S2, S3 exist + * -Compact acquires a replace lock + * -A concurrent appending job publishes a segment S4 which needs to be upgraded to the replace lock's version + * -Compact task processes S1-S4 to create new segments + * -Compact task publishes new segments and carries S4 forward to the new version + * + * This can lead to the data in S4 being duplicated + * + * This TaskAction returns a collection of segments which have data within the specified interval and are marked as + * used, and have been created before a REPLACE lock, if any, was acquired. + * This ensures that a consistent set of segments is returned each time this action is called + */ +public class RetrieveSegmentsToReplaceAction implements TaskAction> +{ + private static final Logger log = new Logger(RetrieveSegmentsToReplaceAction.class); + + @JsonIgnore + private final String dataSource; + + @JsonIgnore + private final Interval interval; + + @JsonCreator + public RetrieveSegmentsToReplaceAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + this.dataSource = dataSource; + this.interval = interval; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @Override + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public Collection perform(Task task, TaskActionToolbox toolbox) + { + // The DruidInputSource can be used to read from one datasource and write to another. + // In such a case, the race condition described in the class-level docs cannot occur, + // and the action can simply fetch all visible segments for the datasource and interval + if (!task.getDataSource().equals(dataSource)) { + return retrieveAllVisibleSegments(toolbox); + } + + final String supervisorId; + if (task instanceof AbstractBatchSubtask) { + supervisorId = ((AbstractBatchSubtask) task).getSupervisorTaskId(); + } else { + supervisorId = task.getId(); + } + + final Set replaceLocksForTask = toolbox + .getTaskLockbox() + .getAllReplaceLocksForDatasource(task.getDataSource()) + .stream() + .filter(lock -> supervisorId.equals(lock.getSupervisorTaskId())) + .collect(Collectors.toSet()); + + // If there are no replace locks for the task, simply fetch all visible segments for the interval + if (replaceLocksForTask.isEmpty()) { + return retrieveAllVisibleSegments(toolbox); + } + + Map>> intervalToCreatedToSegments = new HashMap<>(); + for (Pair segmentAndCreatedDate : + toolbox.getIndexerMetadataStorageCoordinator().retrieveUsedSegmentsAndCreatedDates(dataSource, interval)) { + final DataSegment segment = segmentAndCreatedDate.lhs; + final String created = segmentAndCreatedDate.rhs; + intervalToCreatedToSegments.computeIfAbsent(segment.getInterval(), s -> new HashMap<>()) + .computeIfAbsent(created, c -> new HashSet<>()) + .add(segment); + } + + Set allSegmentsToBeReplaced = new HashSet<>(); + for (final Map.Entry>> entry : intervalToCreatedToSegments.entrySet()) { + final Interval segmentInterval = entry.getKey(); + String lockVersion = null; + for (ReplaceTaskLock replaceLock : replaceLocksForTask) { + if (replaceLock.getInterval().contains(segmentInterval)) { + lockVersion = replaceLock.getVersion(); + } + } + final Map> createdToSegmentsMap = entry.getValue(); + for (Map.Entry> createdAndSegments : createdToSegmentsMap.entrySet()) { + if (lockVersion == null || lockVersion.compareTo(createdAndSegments.getKey()) > 0) { + allSegmentsToBeReplaced.addAll(createdAndSegments.getValue()); + } else { + for (DataSegment segment : createdAndSegments.getValue()) { + log.info("Ignoring segment[%s] as it has created_date[%s] greater than the REPLACE lock version[%s]", + segment.getId(), createdAndSegments.getKey(), lockVersion); + } + } + } + } + + return SegmentTimeline.forSegments(allSegmentsToBeReplaced) + .findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE); + } + + private Collection retrieveAllVisibleSegments(TaskActionToolbox toolbox) + { + return toolbox.getIndexerMetadataStorageCoordinator() + .retrieveUsedSegmentsForInterval(dataSource, interval, Segments.ONLY_VISIBLE); + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RetrieveSegmentsToReplaceAction that = (RetrieveSegmentsToReplaceAction) o; + + if (!dataSource.equals(that.dataSource)) { + return false; + } + return interval.equals(that.interval); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, interval); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "dataSource='" + dataSource + '\'' + + ", interval=" + interval + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 171d53b9cdd6..e251626f8690 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -38,6 +38,7 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), + @JsonSubTypes.Type(name = "retrieveSegmentsToReplace", value = RetrieveSegmentsToReplaceAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. @JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java index f4926864dcbe..088169d3a537 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java @@ -23,11 +23,15 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Optional; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskRunner; +/* This class was added for mm-less ingestion in order to let the peon manage its own location lifecycle by submitting +actions to the overlord. https://github.com/apache/druid/pull/15133 moved this location logic to the overlord itself +so this Action is no longer needed. For backwards compatibility with old peons, this class was left in but can be deprecated +for a later druid release. +*/ +@Deprecated public class UpdateLocationAction implements TaskAction { @JsonIgnore @@ -58,10 +62,6 @@ public TypeReference getReturnTypeReference() @Override public Void perform(Task task, TaskActionToolbox toolbox) { - Optional taskRunner = toolbox.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().updateLocation(task, taskLocation); - } return null; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index fe19b35391e9..cbc469b61333 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -62,6 +62,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -483,13 +484,18 @@ private TaskLockType determineLockType(LockGranularity lockGranularity) return TaskLockType.EXCLUSIVE; } - final String contextLockType = getContextValue(Tasks.TASK_LOCK_TYPE); + final TaskLockType contextTaskLockType = QueryContexts.getAsEnum( + Tasks.TASK_LOCK_TYPE, + getContextValue(Tasks.TASK_LOCK_TYPE), + TaskLockType.class + ); + final TaskLockType lockType; - if (contextLockType == null) { + if (contextTaskLockType == null) { lockType = getContextValue(Tasks.USE_SHARED_LOCK, false) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; } else { - lockType = TaskLockType.valueOf(contextLockType); + lockType = contextTaskLockType; } final IngestionMode ingestionMode = getIngestionMode(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java index e5b6ab1b7312..ea0cb566b2ce 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java @@ -25,13 +25,11 @@ import com.google.common.base.Objects; import com.google.common.base.Preconditions; import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.UpdateLocationAction; import org.apache.druid.indexing.common.actions.UpdateStatusAction; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; @@ -42,14 +40,12 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.BatchIOConfig; -import org.apache.druid.server.DruidNode; import org.joda.time.Interval; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.net.InetAddress; import java.nio.file.Files; import java.nio.file.Paths; import java.util.HashMap; @@ -155,11 +151,6 @@ public String setup(TaskToolbox toolbox) throws Exception FileUtils.mkdirp(attemptDir); reportsFile = new File(attemptDir, "report.json"); statusFile = new File(attemptDir, "status.json"); - InetAddress hostName = InetAddress.getLocalHost(); - DruidNode node = toolbox.getTaskExecutorNode(); - toolbox.getTaskActionClient().submit(new UpdateLocationAction(TaskLocation.create( - hostName.getHostAddress(), node.getPlaintextPort(), node.getTlsPort(), node.isEnablePlaintextPort() - ))); } log.debug("Task setup complete"); return null; @@ -211,7 +202,6 @@ public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception status = new UpdateStatusAction("failure"); } toolbox.getTaskActionClient().submit(status); - toolbox.getTaskActionClient().submit(new UpdateLocationAction(TaskLocation.unknown())); if (reportsFile != null && reportsFile.exists()) { toolbox.getTaskLogPusher().pushTaskReports(id, reportsFile); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractBatchSubtask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractBatchSubtask.java index 37b70c53ed55..e6de0cf25949 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractBatchSubtask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractBatchSubtask.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.TaskResource; @@ -29,16 +30,20 @@ public abstract class AbstractBatchSubtask extends AbstractBatchIndexTask { + private final String supervisorTaskId; + protected AbstractBatchSubtask( String id, @Nullable String groupId, @Nullable TaskResource taskResource, String dataSource, @Nullable Map context, - @Nonnull IngestionMode ingestionMode + @Nonnull IngestionMode ingestionMode, + @Nonnull String supervisorTaskId ) { super(id, groupId, taskResource, dataSource, context, -1, ingestionMode); + this.supervisorTaskId = supervisorTaskId; } /** @@ -46,4 +51,13 @@ protected AbstractBatchSubtask( * This ID is used to identify duplicate work of retry tasks for the same spec. */ public abstract String getSubtaskSpecId(); + + /** + * @return Task ID of the {@code ParallelIndexSupervisorTask} which launched this sub-task. + */ + @JsonProperty + public String getSupervisorTaskId() + { + return supervisorTaskId; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index 625b20517836..698f8c9d0e95 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -69,7 +69,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; - private final String supervisorTaskId; private final String subtaskSpecId; private final ObjectMapper jsonMapper; @@ -95,7 +94,8 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask taskResource, ingestionSchema.getDataSchema(), ingestionSchema.getTuningConfig(), - context + context, + supervisorTaskId ); Preconditions.checkArgument( @@ -107,7 +107,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask this.subtaskSpecId = subtaskSpecId; this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; this.jsonMapper = jsonMapper; } @@ -123,12 +122,6 @@ private ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - @JsonProperty - private String getSupervisorTaskId() - { - return supervisorTaskId; - } - @JsonProperty @Override public String getSubtaskSpecId() @@ -163,7 +156,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception { if (!getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) { return tryTimeChunkLock( - new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), + new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient), getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() ); } else { @@ -274,7 +267,7 @@ private void sendReport(TaskToolbox toolbox, DimensionCardinalityReport report) { final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( - supervisorTaskId, + getSupervisorTaskId(), ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index b2ecd3dc2695..8f03c3bfa55e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -82,7 +82,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; - private final String supervisorTaskId; private final String subtaskSpecId; // For testing @@ -136,7 +135,8 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask taskResource, ingestionSchema.getDataSchema(), ingestionSchema.getTuningConfig(), - context + context, + supervisorTaskId ); Preconditions.checkArgument( @@ -148,7 +148,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask this.subtaskSpecId = subtaskSpecId; this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; this.dedupInputRowFilterSupplier = dedupRowDimValueFilterSupplier; } @@ -164,12 +163,6 @@ private ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - @JsonProperty - private String getSupervisorTaskId() - { - return supervisorTaskId; - } - @JsonProperty @Override public String getSubtaskSpecId() @@ -204,7 +197,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception { if (!getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) { return tryTimeChunkLock( - new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), + new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient), getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() ); } else { @@ -326,7 +319,7 @@ private Map determineDistribution( private void sendReport(TaskToolbox toolbox, DimensionDistributionReport report) { final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( - supervisorTaskId, + getSupervisorTaskId(), ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index b91a6ce3a821..49e3591ff18a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -63,7 +63,6 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask intervalToNumShardsOverride; @@ -96,7 +95,6 @@ public PartialHashSegmentGenerateTask( this.subtaskSpecId = subtaskSpecId; this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; this.intervalToNumShardsOverride = intervalToNumShardsOverride; } @@ -112,12 +110,6 @@ public ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - @JsonProperty - public String getSupervisorTaskId() - { - return supervisorTaskId; - } - @JsonProperty @Override public String getSubtaskSpecId() @@ -158,7 +150,7 @@ public Set getInputSourceResources() public boolean isReady(TaskActionClient taskActionClient) throws Exception { return tryTimeChunkLock( - new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), + new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient), getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() ); } @@ -175,7 +167,7 @@ SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelInd getDataSource(), getSubtaskSpecId(), granularitySpec, - new SupervisorTaskAccess(supervisorTaskId, taskClient), + new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), createHashPartitionAnalysisFromPartitionsSpec( granularitySpec, partitionsSpec, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index 147a1fbf1212..27604eb7e770 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -64,7 +64,6 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask< private static final String PROP_SPEC = "spec"; private static final boolean SKIP_NULL = true; - private final String supervisorTaskId; private final String subtaskSpecId; private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; @@ -98,7 +97,6 @@ public PartialRangeSegmentGenerateTask( this.subtaskSpecId = subtaskSpecId; this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; this.intervalToPartitions = intervalToPartitions; } @@ -131,12 +129,6 @@ public ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - @JsonProperty - public String getSupervisorTaskId() - { - return supervisorTaskId; - } - @JsonProperty @Override public String getSubtaskSpecId() @@ -176,7 +168,7 @@ public Set getInputSourceResources() public boolean isReady(TaskActionClient taskActionClient) throws IOException { return tryTimeChunkLock( - new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), + new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient), getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() ); } @@ -194,7 +186,7 @@ SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelInd getDataSource(), getSubtaskSpecId(), ingestionSchema.getDataSchema().getGranularitySpec(), - new SupervisorTaskAccess(supervisorTaskId, taskClient), + new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), partitionAnalysis ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index dc2a7ef5bf92..e20c7bdbe352 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -94,7 +94,8 @@ abstract class PartialSegmentGenerateTask e taskResource, ingestionSchema.getDataSchema(), ingestionSchema.getTuningConfig(), - context + context, + supervisorTaskId ); Preconditions.checkArgument( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index bb933169d4b7..b59ec65716c2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -78,7 +78,6 @@ abstract class PartialSegmentMergeTask extends PerfectRollu private final PartialSegmentMergeIOConfig ioConfig; private final int numAttempts; - private final String supervisorTaskId; private final String subtaskSpecId; PartialSegmentMergeTask( @@ -101,7 +100,8 @@ abstract class PartialSegmentMergeTask extends PerfectRollu taskResource, dataSchema, tuningConfig, - context + context, + supervisorTaskId ); Preconditions.checkArgument( @@ -111,7 +111,6 @@ abstract class PartialSegmentMergeTask extends PerfectRollu this.subtaskSpecId = subtaskSpecId; this.ioConfig = ioConfig; this.numAttempts = numAttempts; - this.supervisorTaskId = supervisorTaskId; } @JsonProperty @@ -120,12 +119,6 @@ public int getNumAttempts() return numAttempts; } - @JsonProperty - public String getSupervisorTaskId() - { - return supervisorTaskId; - } - @JsonProperty @Override public String getSubtaskSpecId() @@ -151,7 +144,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception } final List locks = toolbox.getTaskActionClient().submit( - new SurrogateAction<>(supervisorTaskId, new LockListAction()) + new SurrogateAction<>(getSupervisorTaskId(), new LockListAction()) ); final Map intervalToVersion = Maps.newHashMapWithExpectedSize(locks.size()); locks.forEach(lock -> { @@ -179,7 +172,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception LOG.info("Fetch took [%s] seconds", fetchTime); final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( - supervisorTaskId, + getSupervisorTaskId(), getTuningConfig().getChatHandlerTimeout(), getTuningConfig().getChatHandlerNumRetries() ); @@ -225,7 +218,7 @@ private Map>> fetchSegmentFiles( ); FileUtils.mkdirp(partitionDir); for (PartitionLocation location : entryPerBucketId.getValue()) { - final File unzippedDir = toolbox.getShuffleClient().fetchSegmentFile(partitionDir, supervisorTaskId, location); + final File unzippedDir = toolbox.getShuffleClient().fetchSegmentFile(partitionDir, getSupervisorTaskId(), location); intervalToUnzippedFiles.computeIfAbsent(interval, k -> new Int2ObjectOpenHashMap<>()) .computeIfAbsent(bucketId, k -> new ArrayList<>()) .add(unzippedDir); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java index 4259922b43ad..3b00f0fedf67 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java @@ -49,10 +49,11 @@ abstract class PerfectRollupWorkerTask extends AbstractBatchSubtask @Nullable TaskResource taskResource, DataSchema dataSchema, ParallelIndexTuningConfig tuningConfig, - @Nullable Map context + @Nullable Map context, + String supervisorTaskId ) { - super(id, groupId, taskResource, dataSchema.getDataSource(), context, IngestionMode.NONE); + super(id, groupId, taskResource, dataSchema.getDataSource(), context, IngestionMode.NONE, supervisorTaskId); Preconditions.checkArgument( tuningConfig.isForceGuaranteedRollup(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 183cde7c66d0..a3bd47d29603 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -118,7 +118,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; - private final String supervisorTaskId; private final String subtaskSpecId; /** @@ -169,7 +168,8 @@ public SinglePhaseSubTask( taskResource, ingestionSchema.getDataSchema().getDataSource(), context, - AbstractTask.computeBatchIngestionMode(ingestionSchema.getIOConfig()) + AbstractTask.computeBatchIngestionMode(ingestionSchema.getIOConfig()), + supervisorTaskId ); if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { @@ -179,7 +179,6 @@ public SinglePhaseSubTask( this.subtaskSpecId = subtaskSpecId; this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; this.missingIntervalsInOverwriteMode = ingestionSchema.getIOConfig().isAppendToExisting() != true && ingestionSchema.getDataSchema() .getGranularitySpec() @@ -217,7 +216,7 @@ public Set getInputSourceResources() public boolean isReady(TaskActionClient taskActionClient) throws IOException { return determineLockGranularityAndTryLock( - new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), + new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient), ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals() ); } @@ -234,12 +233,6 @@ public ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - @JsonProperty - public String getSupervisorTaskId() - { - return supervisorTaskId; - } - @Override @JsonProperty public String getSubtaskSpecId() @@ -272,7 +265,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(toolbox); final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( - supervisorTaskId, + getSupervisorTaskId(), ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java index bf8b4bfb1d51..8056c69901fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java @@ -48,10 +48,9 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.RetrieveSegmentsToReplaceAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.firehose.WindowedSegmentId; -import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Comparators; @@ -552,14 +551,7 @@ public static List> getTimelineForInte } else { try { usedSegments = toolbox.getTaskActionClient() - .submit( - new RetrieveUsedSegmentsAction( - dataSource, - null, - Collections.singletonList(interval), - Segments.ONLY_VISIBLE - ) - ); + .submit(new RetrieveSegmentsToReplaceAction(dataSource, interval)); } catch (IOException e) { LOG.error(e, "Error retrieving the used segments for interval[%s].", interval); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java index ac1fd124ef55..99b0c05b8323 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java @@ -141,11 +141,6 @@ default void updateStatus(Task task, TaskStatus status) // do nothing } - default void updateLocation(Task task, TaskLocation location) - { - // do nothing - } - /** * The maximum number of tasks this TaskRunner can run concurrently. * Can return -1 if this method is not implemented or capacity can't be found. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/UpdateLocationActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/UpdateLocationActionTest.java deleted file mode 100644 index 83aeb382dc0a..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/UpdateLocationActionTest.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.actions; - -import com.google.common.base.Optional; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskRunner; -import org.junit.Test; - -import java.net.InetAddress; -import java.net.UnknownHostException; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class UpdateLocationActionTest -{ - @Test - public void testFlow() throws UnknownHostException - { - // get my task location - InetAddress hostName = InetAddress.getLocalHost(); - TaskLocation myLocation = TaskLocation.create(hostName.getHostAddress(), 1, 2); - UpdateLocationAction action = new UpdateLocationAction(myLocation); - Task task = NoopTask.create(); - TaskActionToolbox toolbox = mock(TaskActionToolbox.class); - TaskRunner runner = mock(TaskRunner.class); - when(toolbox.getTaskRunner()).thenReturn(Optional.of(runner)); - action.perform(task, toolbox); - verify(runner, times(1)).updateLocation(eq(task), eq(myLocation)); - } - - @Test - public void testWithNoTaskRunner() throws UnknownHostException - { - // get my task location - InetAddress hostName = InetAddress.getLocalHost(); - TaskLocation myLocation = TaskLocation.create(hostName.getHostAddress(), 1, 2); - UpdateLocationAction action = new UpdateLocationAction(myLocation); - Task task = NoopTask.create(); - TaskActionToolbox toolbox = mock(TaskActionToolbox.class); - TaskRunner runner = mock(TaskRunner.class); - when(toolbox.getTaskRunner()).thenReturn(Optional.absent()); - action.perform(task, toolbox); - verify(runner, never()).updateStatus(any(), any()); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AbstractTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AbstractTaskTest.java index 5bcadcfb7125..39b0bdfcfc50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AbstractTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AbstractTaskTest.java @@ -109,7 +109,7 @@ public String setup(TaskToolbox toolbox) throws Exception task.run(toolbox); // call it 3 times, once to update location in setup, then one for status and location in cleanup - Mockito.verify(taskActionClient, times(3)).submit(any()); + Mockito.verify(taskActionClient, times(1)).submit(any()); verify(pusher, times(1)).pushTaskReports(eq("myID"), any()); verify(pusher, times(1)).pushTaskStatus(eq("myID"), any()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java index 71b474e77359..b95fd53c74f7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java @@ -111,7 +111,8 @@ PerfectRollupWorkerTask build() null, createDataSchema(granularitySpecInputIntervals), createTuningConfig(forceGuaranteedRollup, partitionsSpec), - null + null, + "supervisor-id" ); } @@ -149,10 +150,11 @@ private static class TestPerfectRollupWorkerTask extends PerfectRollupWorkerTask @Nullable TaskResource taskResource, DataSchema dataSchema, ParallelIndexTuningConfig tuningConfig, - @Nullable Map context + @Nullable Map context, + String supervisorId ) { - super(id, groupId, taskResource, dataSchema, tuningConfig, context); + super(id, groupId, taskResource, dataSchema, tuningConfig, context, supervisorId); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 22f21fb79b62..1c4b6809c387 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.actions.RetrieveSegmentsToReplaceAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -845,6 +846,54 @@ public void testSegmentIsAllocatedAtLatestVersion() verifyIntervalHasVisibleSegments(YEAR_23, segmentV20, segmentV21, segmentV22, segmentV23); } + @Test + public void testSegmentsToReplace() + { + final SegmentIdWithShardSpec pendingSegmentV01 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(SEGMENT_V0, pendingSegmentV01.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV01.getInterval()); + final DataSegment segment1 = asSegment(pendingSegmentV01); + appendTask.commitAppendSegments(segment1); + + final SegmentIdWithShardSpec pendingSegmentV02 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV02.asSegmentId()); + Assert.assertEquals(SEGMENT_V0, pendingSegmentV02.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV02.getInterval()); + + verifyInputSegments(replaceTask, JAN_23, segment1); + + replaceTask.acquireReplaceLockOn(JAN_23); + + final DataSegment segment2 = asSegment(pendingSegmentV02); + appendTask.commitAppendSegments(segment2); + + // Despite segment2 existing, it is not chosen to be replaced because it was created after the tasklock was acquired + verifyInputSegments(replaceTask, JAN_23, segment1); + + replaceTask.releaseLock(JAN_23); + + final SegmentIdWithShardSpec pendingSegmentV03 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV03.asSegmentId()); + Assert.assertNotEquals(pendingSegmentV02.asSegmentId(), pendingSegmentV03.asSegmentId()); + Assert.assertEquals(SEGMENT_V0, pendingSegmentV03.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV03.getInterval()); + final DataSegment segment3 = asSegment(pendingSegmentV03); + appendTask.commitAppendSegments(segment3); + appendTask.releaseLock(JAN_23); + + replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23); + // The new lock was acquired before segment3 was created but it doesn't contain the month's interval + // So, all three segments are chosen + verifyInputSegments(replaceTask, JAN_23, segment1, segment2, segment3); + + replaceTask.releaseLock(FIRST_OF_JAN_23); + // All the segments are chosen when there's no lock + verifyInputSegments(replaceTask, JAN_23, segment1, segment2, segment3); + } + @Nullable private DataSegment findSegmentWith(String version, Map loadSpec, Set segments) { @@ -901,6 +950,23 @@ private void verifySegments(Interval interval, Segments visibility, DataSegment. } } + private void verifyInputSegments(Task task, Interval interval, DataSegment... expectedSegments) + { + try { + final TaskActionClient taskActionClient = taskActionClientFactory.create(task); + Collection allUsedSegments = taskActionClient.submit( + new RetrieveSegmentsToReplaceAction( + WIKI, + interval + ) + ); + Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); + } + catch (IOException e) { + throw new ISE(e, "Error while fetching segments to replace in interval[%s]", interval); + } + } + private TaskToolboxFactory createToolboxFactory( TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 214643affa88..143a74c72cbc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -89,7 +89,7 @@ public List retrieveAllUsedSegments(String dataSource, Segments vis } @Override - public List> retrieveUsedSegmentsAndCreatedDates(String dataSource) + public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval) { return ImmutableList.of(); } diff --git a/pom.xml b/pom.xml index 945cc9d56d9d..e0617ffbcc2c 100644 --- a/pom.xml +++ b/pom.xml @@ -1809,7 +1809,7 @@ com.github.eirslett frontend-maven-plugin - 1.11.3 + 1.14.0 diff --git a/processing/src/main/java/org/apache/druid/data/input/Rows.java b/processing/src/main/java/org/apache/druid/data/input/Rows.java index c10460c86c8a..add6781b81b1 100644 --- a/processing/src/main/java/org/apache/druid/data/input/Rows.java +++ b/processing/src/main/java/org/apache/druid/data/input/Rows.java @@ -25,7 +25,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.math.expr.Evals; import javax.annotation.Nullable; import java.util.Arrays; @@ -60,7 +59,11 @@ public static List toGroupKey(long timeStamp, InputRow inputRow) } /** - * Convert an object to a list of strings. + * Convert an object to a list of strings. This function translates single value nulls into an empty list, and any + * nulls inside of a list or array into the string "null". Do not use this method if you don't want this behavior, + * but note that many implementations of {@link InputRow#getDimension(String)} do use this method, so it is + * recommended to use {@link InputRow#getRaw(String)} if you want the actual value without this coercion. For legacy + * reasons, some stuff counts on this incorrect behavior, (such as {@link Rows#toGroupKey(long, InputRow)}). */ public static List objectToStrings(final Object inputValue) { @@ -73,7 +76,7 @@ public static List objectToStrings(final Object inputValue) // convert byte[] to base64 encoded string return Collections.singletonList(StringUtils.encodeBase64String((byte[]) inputValue)); } else if (inputValue instanceof Object[]) { - return Arrays.stream((Object[]) inputValue).map(Evals::asString).collect(Collectors.toList()); + return Arrays.stream((Object[]) inputValue).map(String::valueOf).collect(Collectors.toList()); } else { return Collections.singletonList(String.valueOf(inputValue)); } diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java index 48f2a5117ee8..ec7de095e12c 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java @@ -37,6 +37,7 @@ public ColumnValueSelector makeColumnValueSelector( { return new NumericArrayFieldSelector(memory, fieldPointer) { + private static final int FIELD_SIZE = Byte.BYTES + Double.BYTES; final SettableFieldPointer fieldPointer = new SettableFieldPointer(); final ColumnValueSelector columnValueSelector = DoubleFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); @@ -45,7 +46,7 @@ public ColumnValueSelector makeColumnValueSelector( @Override public Double getIndividualValueAtMemory(long position) { - fieldPointer.setPosition(position); + fieldPointer.setPositionAndLength(position, FIELD_SIZE); if (columnValueSelector.isNull()) { return null; } @@ -55,7 +56,7 @@ public Double getIndividualValueAtMemory(long position) @Override public int getIndividualFieldSize() { - return Byte.BYTES + Double.BYTES; + return FIELD_SIZE; } }; } diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java index fcbe407bdb26..e97af071824e 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java @@ -37,6 +37,7 @@ public ColumnValueSelector makeColumnValueSelector( { return new NumericArrayFieldSelector(memory, fieldPointer) { + private static final int FIELD_SIZE = Byte.BYTES + Float.BYTES; final SettableFieldPointer fieldPointer = new SettableFieldPointer(); final ColumnValueSelector columnValueSelector = FloatFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); @@ -45,7 +46,7 @@ public ColumnValueSelector makeColumnValueSelector( @Override public Float getIndividualValueAtMemory(long position) { - fieldPointer.setPosition(position); + fieldPointer.setPositionAndLength(position, FIELD_SIZE); if (columnValueSelector.isNull()) { return null; } @@ -55,7 +56,7 @@ public Float getIndividualValueAtMemory(long position) @Override public int getIndividualFieldSize() { - return Byte.BYTES + Float.BYTES; + return FIELD_SIZE; } }; } diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java index b52b39d13c47..8f7578c07d38 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java @@ -37,6 +37,7 @@ public ColumnValueSelector makeColumnValueSelector( { return new NumericArrayFieldSelector(memory, fieldPointer) { + private static final int FIELD_SIZE = Byte.BYTES + Long.BYTES; final SettableFieldPointer fieldPointer = new SettableFieldPointer(); final ColumnValueSelector columnValueSelector = LongFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); @@ -45,7 +46,7 @@ public ColumnValueSelector makeColumnValueSelector( @Override public Long getIndividualValueAtMemory(long position) { - fieldPointer.setPosition(position); + fieldPointer.setPositionAndLength(position, FIELD_SIZE); if (columnValueSelector.isNull()) { return null; } @@ -55,7 +56,7 @@ public Long getIndividualValueAtMemory(long position) @Override public int getIndividualFieldSize() { - return Byte.BYTES + Long.BYTES; + return FIELD_SIZE; } }; } diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java index 1871aef06e08..f15361d47ea1 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java @@ -25,8 +25,6 @@ import org.apache.druid.segment.ColumnValueSelector; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.List; /** * Base implementation of the column value selector that the concrete numeric field reader implementations inherit from. @@ -66,12 +64,8 @@ public abstract class NumericArrayFieldSelector impl /** * Value of the row at the location beginning at {@link #currentFieldPosition} */ - private final List currentRow = new ArrayList<>(); - - /** - * Nullity of the row at the location beginning at {@link #currentFieldPosition} - */ - private boolean currentRowIsNull; + @Nullable + private Number[] currentRow = null; public NumericArrayFieldSelector(final Memory memory, final ReadableFieldPointer fieldPointer) { @@ -89,13 +83,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public Object getObject() { - final List currentArray = computeCurrentArray(); - - if (currentArray == null) { - return null; - } - - return currentArray.toArray(); + return computeCurrentArray(); } @Override @@ -143,34 +131,29 @@ public boolean isNull() public abstract int getIndividualFieldSize(); @Nullable - private List computeCurrentArray() + private Number[] computeCurrentArray() { final long fieldPosition = fieldPointer.position(); + final long fieldLength = fieldPointer.length(); if (fieldPosition != currentFieldPosition) { - updateCurrentArray(fieldPosition); + updateCurrentArray(fieldPosition, fieldLength); } this.currentFieldPosition = fieldPosition; - - if (currentRowIsNull) { - return null; - } return currentRow; - } - private void updateCurrentArray(final long fieldPosition) + private void updateCurrentArray(final long fieldPosition, final long fieldLength) { - currentRow.clear(); - currentRowIsNull = false; + currentRow = null; long position = fieldPosition; long limit = memory.getCapacity(); // Check the first byte, and if it is null, update the current value to null and return if (isNull()) { - currentRowIsNull = true; + // Already set the currentRow to null return; } @@ -179,9 +162,13 @@ private void updateCurrentArray(final long fieldPosition) position++; } + int numElements = numElements(fieldLength); + currentRow = new Number[numElements]; + // Sanity check, to make sure that we see the rowTerminator at the end boolean rowTerminatorSeen = false; + int curElement = 0; while (position < limit) { final byte kind = memory.getByte(position); @@ -193,12 +180,26 @@ private void updateCurrentArray(final long fieldPosition) // If terminator not seen, then read the field at that location, and increment the position by the element's field // size to read the next element. - currentRow.add(getIndividualValueAtMemory(position)); + currentRow[curElement] = getIndividualValueAtMemory(position); position += getIndividualFieldSize(); + curElement++; } - if (!rowTerminatorSeen) { + if (!rowTerminatorSeen || curElement != numElements) { throw DruidException.defensive("Unexpected end of field"); } } + + int numElements(long fieldSize) + { + if (fieldSize <= 1) { + throw DruidException.defensive("fieldSize should be greater than 1 for non null array elements"); + } + // Remove one byte for the nullity byte, and one for the array terminator + long cumulativeFieldSize = fieldSize - Byte.BYTES - Byte.BYTES; + if (cumulativeFieldSize % getIndividualFieldSize() != 0) { + throw DruidException.defensive("cumulativeFieldSize should be a multiple of the individual fieldSize"); + } + return Math.toIntExact(cumulativeFieldSize / getIndividualFieldSize()); + } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/ReadableFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/ReadableFieldPointer.java index 29412b507af8..09de051a691e 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/ReadableFieldPointer.java +++ b/processing/src/main/java/org/apache/druid/frame/field/ReadableFieldPointer.java @@ -31,4 +31,9 @@ public interface ReadableFieldPointer * Starting position of the field. */ long position(); + + /** + * Length of the field. + */ + long length(); } diff --git a/processing/src/main/java/org/apache/druid/frame/field/RowMemoryFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/RowMemoryFieldPointer.java index 881c9adb04f9..d90eb3eaa22a 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/RowMemoryFieldPointer.java +++ b/processing/src/main/java/org/apache/druid/frame/field/RowMemoryFieldPointer.java @@ -24,6 +24,10 @@ /** * A {@link ReadableFieldPointer} that is derived from a row-based frame. + * + * Returns the position and the length of a field at a particular position for the row that the rowPointer is pointing + * to at the time. It caches the values of the position and the length based on position of the rowPointer. + * This method is not thread-safe */ public class RowMemoryFieldPointer implements ReadableFieldPointer { @@ -32,6 +36,16 @@ public class RowMemoryFieldPointer implements ReadableFieldPointer private final int fieldNumber; private final int fieldCount; + // Caching of position() calls + private long rowWithCachedPosition = -1L; + private long cachedPosition = -1L; + + // Caching of length() calls + // We cache the length() calls separately, because not all field types call length(), therefore it's wasteful to + // compute and cache length() if we are not reading it + private long rowWithCachedLength = -1L; + private long cachedLength = -1L; + public RowMemoryFieldPointer( final Memory memory, final ReadableFrameRowPointer rowPointer, @@ -47,6 +61,63 @@ public RowMemoryFieldPointer( @Override public long position() + { + updatePosition(); + return cachedPosition; + } + + @Override + public long length() + { + updatePositionAndLength(); + return cachedLength; + } + + private void updatePosition() + { + long rowPointerPosition = rowPointer.position(); + if (rowPointerPosition == rowWithCachedPosition) { + return; + } + // Update the cached position for position() + rowWithCachedPosition = rowPointerPosition; + + // Update the start position + cachedPosition = startPosition(fieldNumber); + } + + // Not all field types call length(), and therefore there's no need to cache the length of the field. This method + // updates both the position and the length. + private void updatePositionAndLength() + { + updatePosition(); + + // rowPointerPosition = rowPointer.position() = rowWithCachedPosition, since that was updated in the call to update + // position above + long rowPointerPosition = rowWithCachedPosition; + + if (rowPointerPosition == rowWithCachedLength) { + return; + } + // Update the cached position for length() + rowWithCachedLength = rowPointerPosition; + + if (fieldNumber == fieldCount - 1) { + // If the field is the last field in the row, then the length of the field would be the end of the row minus the + // start position of the field. End of the row is the start of the row plus the length of the row. + cachedLength = (rowPointerPosition + rowPointer.length()) - cachedPosition; + } else { + // Else the length of the field would be the difference between the start position of the given field and + // the subsequent field + cachedLength = startPosition(fieldNumber + 1) - cachedPosition; + } + } + + /** + * Given a fieldNumber, computes the start position of the field. Requires a memory access to read the start position, + * therefore callers should cache the value for better efficiency. + */ + private long startPosition(int fieldNumber) { if (fieldNumber == 0) { // First field starts after the field end pointers -- one integer per field. diff --git a/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java index d26f84f251de..0242b412ca08 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java +++ b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java @@ -20,16 +20,20 @@ package org.apache.druid.frame.field; /** - * A simple {@link ReadableFieldPointer} that returns the position that was set on its object. + * A simple {@link ReadableFieldPointer} that returns the position and the length that was set on its object. */ public class SettableFieldPointer implements ReadableFieldPointer { - long position = 0; + long length = -1; - public void setPosition(long position) + /** + * Sets the position and the length to be returned when interface's methods are called. + */ + public void setPositionAndLength(long position, long length) { this.position = position; + this.length = length; } @Override @@ -37,4 +41,10 @@ public long position() { return position; } + + @Override + public long length() + { + return length; + } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/ReadableFrameRowPointer.java b/processing/src/main/java/org/apache/druid/frame/segment/row/ReadableFrameRowPointer.java index 714195396c8e..8276e26b28c3 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/ReadableFrameRowPointer.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/ReadableFrameRowPointer.java @@ -30,7 +30,13 @@ */ public interface ReadableFrameRowPointer { + /** + * Position of the start of the row relative to the start of the Frame + */ long position(); + /** + * Length of the row (in bytes) + */ long length(); } diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 4e4a3b10ec94..95c7b78862b9 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -51,7 +51,9 @@ @JsonSubTypes.Type(name = "false", value = FalseDimFilter.class), @JsonSubTypes.Type(name = "null", value = NullFilter.class), @JsonSubTypes.Type(name = "equals", value = EqualityFilter.class), - @JsonSubTypes.Type(name = "range", value = RangeFilter.class) + @JsonSubTypes.Type(name = "range", value = RangeFilter.class), + @JsonSubTypes.Type(name = "isfalse", value = IsFalseDimFilter.class), + @JsonSubTypes.Type(name = "istrue", value = IsTrueDimFilter.class) }) public interface DimFilter extends Cacheable { diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java index 27a0581d4752..ed03efac38a6 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java @@ -58,6 +58,8 @@ public class DimFilterUtils static final byte EQUALS_CACHE_ID = 0x13; static final byte RANGE_CACHE_ID = 0x14; + static final byte IS_FILTER_BOOLEAN_FILTER_CACHE_ID = 0x15; + public static final byte STRING_SEPARATOR = (byte) 0xFF; diff --git a/processing/src/main/java/org/apache/druid/query/filter/IsBooleanDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/IsBooleanDimFilter.java new file mode 100644 index 000000000000..4b766de3af92 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/IsBooleanDimFilter.java @@ -0,0 +1,120 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.RangeSet; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.segment.filter.IsBooleanFilter; + +import java.util.Objects; +import java.util.Set; + +/** + * Abstract SQL three-value logic wrapper for some child {@link DimFilter} to implement '{filter} IS TRUE' and + * '{filter} IS FALSE'. + * + * @see IsTrueDimFilter - IS TRUE + * @see IsFalseDimFilter - IS FALSE + * @see IsBooleanFilter - actual filtering logic + */ +public abstract class IsBooleanDimFilter extends AbstractOptimizableDimFilter +{ + private final DimFilter field; + private final boolean isTrue; + + public IsBooleanDimFilter( + DimFilter field, + boolean isTrue + ) + { + if (field == null) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("IS %s operator requires a non-null filter for field", isTrue ? "TRUE" : "FALSE"); + } + this.field = field; + this.isTrue = isTrue; + } + + @JsonProperty("field") + public DimFilter getField() + { + return field; + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(DimFilterUtils.IS_FILTER_BOOLEAN_FILTER_CACHE_ID).appendBoolean(isTrue) + .appendCacheable(field) + .build(); + } + + @Override + public Filter toFilter() + { + return new IsBooleanFilter(field.toFilter(), isTrue); + } + + @Override + public RangeSet getDimensionRangeSet(String dimension) + { + return null; + } + + @Override + public Set getRequiredColumns() + { + return field.getRequiredColumns(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IsBooleanDimFilter that = (IsBooleanDimFilter) o; + + if (field != null ? !field.equals(that.field) : that.field != null) { + return false; + } + + return isTrue == that.isTrue; + } + + @Override + public int hashCode() + { + return Objects.hash(field, isTrue); + } + + @Override + public String toString() + { + return "(" + field + ") IS " + (isTrue ? "TRUE" : "FALSE"); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PeonPhaseTest.java b/processing/src/main/java/org/apache/druid/query/filter/IsFalseDimFilter.java similarity index 53% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PeonPhaseTest.java rename to processing/src/main/java/org/apache/druid/query/filter/IsFalseDimFilter.java index 3f6bd71312be..7e674869a883 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PeonPhaseTest.java +++ b/processing/src/main/java/org/apache/druid/query/filter/IsFalseDimFilter.java @@ -17,27 +17,29 @@ * under the License. */ -package org.apache.druid.k8s.overlord.common; +package org.apache.druid.query.filter; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.PodStatus; -import org.junit.jupiter.api.Test; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class PeonPhaseTest +public class IsFalseDimFilter extends IsBooleanDimFilter { + public static IsFalseDimFilter of(DimFilter field) + { + return new IsFalseDimFilter(field); + } + + @JsonCreator + public IsFalseDimFilter( + @JsonProperty("field") DimFilter field + ) + { + super(field, false); + } - @Test - void testGetPhaseForToMakeCoverageHappy() + @Override + public DimFilter optimize() { - Pod pod = mock(Pod.class); - PodStatus status = mock(PodStatus.class); - when(status.getPhase()).thenReturn("Succeeded"); - when(pod.getStatus()).thenReturn(status); - assertEquals(PeonPhase.UNKNOWN, PeonPhase.getPhaseFor(null)); - assertEquals(PeonPhase.SUCCEEDED, PeonPhase.getPhaseFor(pod)); + return new IsFalseDimFilter(getField().optimize()); } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/IsTrueDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/IsTrueDimFilter.java new file mode 100644 index 000000000000..a61897f10f0f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/IsTrueDimFilter.java @@ -0,0 +1,45 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class IsTrueDimFilter extends IsBooleanDimFilter +{ + public static IsTrueDimFilter of(DimFilter field) + { + return new IsTrueDimFilter(field); + } + + @JsonCreator + public IsTrueDimFilter( + @JsonProperty("field") DimFilter field + ) + { + super(field, true); + } + + @Override + public DimFilter optimize() + { + return new IsTrueDimFilter(getField().optimize()); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 5db2dd414e03..a9880c705f9c 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -126,13 +126,21 @@ public ValueMatcher makeValueMatcher(@Nullable String value) @Override public boolean matches(boolean includeUnknown) { - return includeUnknown; + // don't match anything, except for null values when includeUnknown is true + if (includeUnknown) { + if (indexedIntsForCurrentRow == null || indexedIntsForCurrentRow.size() <= 0) { + return true; + } + final int rowId = indexedIntsForCurrentRow.get(index); + return lookupName(rowId) == null; + } + return false; } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - + inspector.visit("selector", dimSelector); } }; } @@ -155,7 +163,7 @@ public boolean matches(boolean includeUnknown) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - dimSelector.inspectRuntimeShape(inspector); + inspector.visit("selector", dimSelector); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java index 6d2bfab4b6a3..1208865d4ce1 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java @@ -186,7 +186,7 @@ public boolean matches(boolean includeUnknown) return Arrays.stream(dResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((double) o)); } } - return (includeUnknown && eval.value() == null) || eval.asBoolean(); + return eval.asBoolean(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java new file mode 100644 index 000000000000..ddf3972ccff8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java @@ -0,0 +1,210 @@ +/* + * 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.filter; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.filter.vector.BaseVectorValueMatcher; +import org.apache.druid.query.filter.vector.ReadableVectorMatch; +import org.apache.druid.query.filter.vector.VectorMatch; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * SQL three-value logic wrapper for some child {@link Filter} to implement '{filter} IS TRUE' and + * '{filter} IS FALSE'. Primarily useful when living beneath a {@link NotFilter} because this filter purposely ignores + * the value of {@code includeUnknown} and so always correctly only returns values that definitely match or do not match + * the filter to produce correct results for '{filter} IS NOT TRUE' and '{filter} IS NOT FALSE'. This filter is a + * relatively thin wrapper, so should be relatively harmless if used without a 'NOT' filter. + * + * @see org.apache.druid.query.filter.IsBooleanDimFilter + * @see org.apache.druid.query.filter.IsTrueDimFilter + * @see org.apache.druid.query.filter.IsFalseDimFilter + */ +public class IsBooleanFilter implements Filter +{ + private final Filter baseFilter; + private final boolean isTrue; + + public IsBooleanFilter(Filter baseFilter, boolean isTrue) + { + this.baseFilter = baseFilter; + this.isTrue = isTrue; + } + + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + final BitmapColumnIndex baseIndex = baseFilter.getBitmapColumnIndex(selector); + if (baseIndex != null && (isTrue || baseIndex.getIndexCapabilities().isInvertible())) { + return new BitmapColumnIndex() + { + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); + @Override + public ColumnIndexCapabilities getIndexCapabilities() + { + return baseIndex.getIndexCapabilities(); + } + + @Override + public double estimateSelectivity(int totalRows) + { + return 1. - baseFilter.estimateSelectivity(selector); + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) + { + if (isTrue) { + return baseIndex.computeBitmapResult(bitmapResultFactory, false); + } + return bitmapResultFactory.complement( + baseIndex.computeBitmapResult(bitmapResultFactory, useThreeValueLogic), + selector.getNumRows() + ); + } + }; + } + return null; + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + final ValueMatcher baseMatcher = baseFilter.makeMatcher(factory); + + return new ValueMatcher() + { + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); + @Override + public boolean matches(boolean includeUnknown) + { + if (isTrue) { + return baseMatcher.matches(false); + } + return !baseMatcher.matches(useThreeValueLogic); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseMatcher", baseMatcher); + } + }; + } + + @Override + public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory factory) + { + final VectorValueMatcher baseMatcher = baseFilter.makeVectorMatcher(factory); + + return new BaseVectorValueMatcher(baseMatcher) + { + private final VectorMatch scratch = VectorMatch.wrap(new int[factory.getMaxVectorSize()]); + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); + + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) + { + if (isTrue) { + return baseMatcher.match(mask, false); + } + final ReadableVectorMatch baseMatch = baseMatcher.match(mask, useThreeValueLogic); + + scratch.copyFrom(mask); + scratch.removeAll(baseMatch); + assert scratch.isValid(mask); + return scratch; + } + }; + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return baseFilter.canVectorizeMatcher(inspector); + } + + @Override + public Set getRequiredColumns() + { + return baseFilter.getRequiredColumns(); + } + + @Override + public boolean supportsRequiredColumnRewrite() + { + return baseFilter.supportsRequiredColumnRewrite(); + } + + @Override + public Filter rewriteRequiredColumns(Map columnRewrites) + { + return new IsBooleanFilter(baseFilter.rewriteRequiredColumns(columnRewrites), isTrue); + } + + @Override + public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector) + { + return baseFilter.supportsSelectivityEstimation(columnSelector, indexSelector); + } + + @Override + public String toString() + { + return StringUtils.format("(%s) IS %s", baseFilter, isTrue ? "TRUE" : "FALSE"); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IsBooleanFilter isFilter = (IsBooleanFilter) o; + return Objects.equals(baseFilter, isFilter.baseFilter); + } + + @Override + public int hashCode() + { + // to return a different hash from baseFilter + return Objects.hash(1, baseFilter, isTrue); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/ComplexFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/ComplexFieldReaderTest.java index 598e7d28edc1..d4970d9bd0cd 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/ComplexFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/ComplexFieldReaderTest.java @@ -123,7 +123,7 @@ public void test_makeColumnValueSelector_null() writeToMemory(null); final ColumnValueSelector readSelector = - new ComplexFieldReader(SERDE).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new ComplexFieldReader(SERDE).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertNull(readSelector.getObject()); } @@ -134,7 +134,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory("foo"); final ColumnValueSelector readSelector = - new ComplexFieldReader(SERDE).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new ComplexFieldReader(SERDE).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals("foo", readSelector.getObject()); } diff --git a/processing/src/main/java/org/apache/druid/frame/field/ConstantFieldPointer.java b/processing/src/test/java/org/apache/druid/frame/field/ConstantFieldPointer.java similarity index 85% rename from processing/src/main/java/org/apache/druid/frame/field/ConstantFieldPointer.java rename to processing/src/test/java/org/apache/druid/frame/field/ConstantFieldPointer.java index 17ff1058794b..0079246c2623 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/ConstantFieldPointer.java +++ b/processing/src/test/java/org/apache/druid/frame/field/ConstantFieldPointer.java @@ -22,10 +22,12 @@ public class ConstantFieldPointer implements ReadableFieldPointer { private final long position; + private final long length; - public ConstantFieldPointer(long position) + public ConstantFieldPointer(long position, long length) { this.position = position; + this.length = length; } @Override @@ -33,4 +35,10 @@ public long position() { return position; } + + @Override + public long length() + { + return length; + } } diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java index 6381138f62db..c931b7aac904 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java @@ -149,10 +149,13 @@ public void test_isNull_arrayWithSingleNullElement() @Test public void test_makeColumnValueSelector_null() { - writeToMemory(null, MEMORY_POSITION); + long sz = writeToMemory(null, MEMORY_POSITION); final ColumnValueSelector readSelector = - new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new DoubleArrayFieldReader().makeColumnValueSelector( + memory, + new ConstantFieldPointer(MEMORY_POSITION, sz) + ); Assert.assertTrue(readSelector.isNull()); } @@ -160,10 +163,14 @@ public void test_makeColumnValueSelector_null() @Test public void test_makeColumnValueSelector_aValue() { - writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + long sz = writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); final ColumnValueSelector readSelector = - new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new DoubleArrayFieldReader() + .makeColumnValueSelector( + memory, + new ConstantFieldPointer(MEMORY_POSITION, sz) + ); assertResults(DOUBLES_LIST_1, readSelector.getObject()); } @@ -172,15 +179,15 @@ public void test_makeColumnValueSelector_aValue() public void test_makeColumnValueSelector_multipleValues() { long sz = writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); - writeToMemory(DOUBLES_ARRAY_2, MEMORY_POSITION + sz); - IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); - + long sz2 = writeToMemory(DOUBLES_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer( + ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz), + ImmutableList.of(sz, sz2) + ); final ColumnValueSelector readSelector = new DoubleArrayFieldReader().makeColumnValueSelector(memory, pointer); - pointer.setPointer(0); assertResults(DOUBLES_LIST_1, readSelector.getObject()); - pointer.setPointer(1); assertResults(DOUBLES_LIST_2, readSelector.getObject()); } @@ -188,10 +195,11 @@ public void test_makeColumnValueSelector_multipleValues() @Test public void test_makeColumnValueSelector_emptyArray() { - writeToMemory(new Object[]{}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new DoubleArrayFieldReader() + .makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(Collections.emptyList(), readSelector.getObject()); } @@ -199,10 +207,13 @@ public void test_makeColumnValueSelector_emptyArray() @Test public void test_makeColumnValueSelector_arrayWithSingleNullElement() { - writeToMemory(new Object[]{null}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{null}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new DoubleArrayFieldReader().makeColumnValueSelector( + memory, + new ConstantFieldPointer(MEMORY_POSITION, sz) + ); assertResults(Collections.singletonList(null), readSelector.getObject()); } diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java index a18b7cc70ec9..c4a300b5bc3c 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java @@ -89,7 +89,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final ColumnValueSelector readSelector = - DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -104,7 +104,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1d); final ColumnValueSelector readSelector = - DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(5.1d, readSelector.getObject()); } @@ -115,7 +115,8 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final DimensionSelector readSelector = - DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -149,7 +150,8 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1d); final DimensionSelector readSelector = - DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -178,7 +180,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() final DimensionSelector readSelector = DoubleFieldReader.forPrimitive().makeDimensionSelector( memory, - new ConstantFieldPointer(MEMORY_POSITION), + new ConstantFieldPointer(MEMORY_POSITION, -1), new SubstringDimExtractionFn(1, null) ); diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java index e61e40db1cb1..3c404ed28dd9 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java @@ -150,10 +150,10 @@ public void test_isNull_arrayWithSingleNullElement() @Test public void test_makeColumnValueSelector_null() { - writeToMemory(null, MEMORY_POSITION); + long sz = writeToMemory(null, MEMORY_POSITION); final ColumnValueSelector readSelector = - new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); Assert.assertTrue(readSelector.isNull()); } @@ -161,10 +161,10 @@ public void test_makeColumnValueSelector_null() @Test public void test_makeColumnValueSelector_aValue() { - writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + long sz = writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); final ColumnValueSelector readSelector = - new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(FLOATS_LIST_1, readSelector.getObject()); } @@ -173,15 +173,15 @@ public void test_makeColumnValueSelector_aValue() public void test_makeColumnValueSelector_multipleValues() { long sz = writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); - writeToMemory(FLOATS_ARRAY_2, MEMORY_POSITION + sz); - IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); - + long sz2 = writeToMemory(FLOATS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer( + ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz), + ImmutableList.of(sz, sz2) + ); final ColumnValueSelector readSelector = new FloatArrayFieldReader().makeColumnValueSelector(memory, pointer); - pointer.setPointer(0); assertResults(FLOATS_LIST_1, readSelector.getObject()); - pointer.setPointer(1); assertResults(FLOATS_LIST_2, readSelector.getObject()); } @@ -189,10 +189,10 @@ public void test_makeColumnValueSelector_multipleValues() @Test public void test_makeColumnValueSelector_emptyArray() { - writeToMemory(new Object[]{}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(Collections.emptyList(), readSelector.getObject()); } @@ -200,10 +200,10 @@ public void test_makeColumnValueSelector_emptyArray() @Test public void test_makeColumnValueSelector_arrayWithSingleNullElement() { - writeToMemory(new Object[]{null}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{null}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(Collections.singletonList(null), readSelector.getObject()); } diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java index 6bae52f1c50d..f25845f9f4ed 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java @@ -89,7 +89,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final ColumnValueSelector readSelector = - FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -104,7 +104,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1f); final ColumnValueSelector readSelector = - FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(5.1f, readSelector.getObject()); } @@ -115,7 +115,8 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final DimensionSelector readSelector = - FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -149,7 +150,8 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1f); final DimensionSelector readSelector = - FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -178,7 +180,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() final DimensionSelector readSelector = FloatFieldReader.forPrimitive().makeDimensionSelector( memory, - new ConstantFieldPointer(MEMORY_POSITION), + new ConstantFieldPointer(MEMORY_POSITION, -1), new SubstringDimExtractionFn(1, null) ); diff --git a/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java index 1e115f48e3c5..22fdbacd1b71 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java +++ b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java @@ -30,11 +30,13 @@ public class IndexArrayFieldPointer implements ReadableFieldPointer { private final LongArrayList indices; + private final LongArrayList lengths; private int pointer = 0; - public IndexArrayFieldPointer(final List indices) + public IndexArrayFieldPointer(final List indices, final List lengths) { this.indices = new LongArrayList(indices); + this.lengths = new LongArrayList(lengths); } private int numIndices() @@ -53,4 +55,10 @@ public long position() { return indices.getLong(pointer); } + + @Override + public long length() + { + return lengths.getLong(pointer); + } } diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java index aa34cd6afaf3..f679bb2b3ed9 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java @@ -126,10 +126,10 @@ public void test_isNull_arrayWithSingleNullElement() @Test public void test_makeColumnValueSelector_null() { - writeToMemory(null, MEMORY_POSITION); + long sz = writeToMemory(null, MEMORY_POSITION); final ColumnValueSelector readSelector = - new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); Assert.assertTrue(readSelector.isNull()); } @@ -137,10 +137,10 @@ public void test_makeColumnValueSelector_null() @Test public void test_makeColumnValueSelector_aValue() { - writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + long sz = writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); final ColumnValueSelector readSelector = - new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(LONGS_LIST_1, readSelector.getObject()); } @@ -149,15 +149,15 @@ public void test_makeColumnValueSelector_aValue() public void test_makeColumnValueSelector_multipleValues() { long sz = writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); - writeToMemory(LONGS_ARRAY_2, MEMORY_POSITION + sz); - IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); - + long sz2 = writeToMemory(LONGS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer( + ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz), + ImmutableList.of(sz, sz2) + ); final ColumnValueSelector readSelector = new LongArrayFieldReader().makeColumnValueSelector(memory, pointer); - pointer.setPointer(0); assertResults(LONGS_LIST_1, readSelector.getObject()); - pointer.setPointer(1); assertResults(LONGS_LIST_2, readSelector.getObject()); } @@ -165,10 +165,10 @@ public void test_makeColumnValueSelector_multipleValues() @Test public void test_makeColumnValueSelector_emptyArray() { - writeToMemory(new Object[]{}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(Collections.emptyList(), readSelector.getObject()); } @@ -176,10 +176,10 @@ public void test_makeColumnValueSelector_emptyArray() @Test public void test_makeColumnValueSelector_arrayWithSingleNullElement() { - writeToMemory(new Object[]{null}, MEMORY_POSITION); + long sz = writeToMemory(new Object[]{null}, MEMORY_POSITION); final ColumnValueSelector readSelector = - new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, sz)); assertResults(Collections.singletonList(null), readSelector.getObject()); } diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java index 643846ee0e3a..8a201394083d 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java @@ -89,7 +89,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final ColumnValueSelector readSelector = - LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -104,7 +104,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5L); final ColumnValueSelector readSelector = - LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(5L, readSelector.getObject()); } @@ -115,7 +115,8 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final DimensionSelector readSelector = - LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -149,7 +150,8 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5L); final DimensionSelector readSelector = - LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive() + .makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -178,7 +180,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() final DimensionSelector readSelector = LongFieldReader.forPrimitive().makeDimensionSelector( memory, - new ConstantFieldPointer(MEMORY_POSITION), + new ConstantFieldPointer(MEMORY_POSITION, -1), new SubstringDimExtractionFn(1, null) ); diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringArrayFieldWriterTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringArrayFieldWriterTest.java index 63dd03d48bf0..02d4d44cbfcc 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringArrayFieldWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringArrayFieldWriterTest.java @@ -140,7 +140,7 @@ private List readFromMemory(final long written) final FieldReader fieldReader = FieldReaders.create("columnNameDoesntMatterHere", ColumnType.STRING_ARRAY); final ColumnValueSelector selector = - fieldReader.makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + fieldReader.makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); final Object o = selector.getObject(); //noinspection rawtypes,unchecked diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java index a682e658ca76..04296cb78c3c 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java @@ -143,9 +143,9 @@ public void test_makeColumnValueSelector_singleString_notArray() writeToMemory(Collections.singletonList("foo")); final ColumnValueSelector readSelector = - new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); final ColumnValueSelector readSelectorAsArray = - new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals("foo", readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo"}, (Object[]) readSelectorAsArray.getObject()); @@ -157,9 +157,9 @@ public void test_makeColumnValueSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final ColumnValueSelector readSelector = - new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); final ColumnValueSelector readSelectorAsArray = - new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertEquals(ImmutableList.of("foo", "bar"), readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo", "bar"}, (Object[]) readSelectorAsArray.getObject()); @@ -171,9 +171,9 @@ public void test_makeColumnValueSelector_null() writeToMemory(Collections.singletonList(null)); final ColumnValueSelector readSelector = - new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); final ColumnValueSelector readSelectorAsArray = - new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(new Object[]{null}, (Object[]) readSelectorAsArray.getObject()); @@ -185,9 +185,9 @@ public void test_makeColumnValueSelector_empty() writeToMemory(Collections.emptyList()); final ColumnValueSelector readSelector = - new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); final ColumnValueSelector readSelectorAsArray = - new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(ObjectArrays.EMPTY_ARRAY, (Object[]) readSelectorAsArray.getObject()); @@ -200,7 +200,11 @@ public void test_makeDimensionSelector_multiString_asArray() final IllegalStateException e = Assert.assertThrows( IllegalStateException.class, - () -> new StringArrayFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null) + () -> new StringArrayFieldReader().makeDimensionSelector( + memory, + new ConstantFieldPointer(MEMORY_POSITION, -1), + null + ) ); MatcherAssert.assertThat( @@ -215,7 +219,7 @@ public void test_makeDimensionSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final DimensionSelector readSelector = - new StringFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + new StringFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -247,7 +251,7 @@ public void test_makeDimensionSelector_multiString_withExtractionFn() final DimensionSelector readSelector = new StringFieldReader().makeDimensionSelector( memory, - new ConstantFieldPointer(MEMORY_POSITION), + new ConstantFieldPointer(MEMORY_POSITION, -1), new SubstringDimExtractionFn(1, null) ); diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldWriterTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldWriterTest.java index 12bbf8238bf2..f44b69e2810f 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldWriterTest.java @@ -184,7 +184,7 @@ private Object[] readFromMemory(final long written) final FieldReader fieldReader = FieldReaders.create("columnNameDoesntMatterHere", ColumnType.STRING_ARRAY); final ColumnValueSelector selector = - fieldReader.makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + fieldReader.makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION, -1)); return (Object[]) selector.getObject(); } diff --git a/processing/src/test/java/org/apache/druid/query/filter/IsBooleanDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/IsBooleanDimFilterTest.java new file mode 100644 index 000000000000..7c4be474efef --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/filter/IsBooleanDimFilterTest.java @@ -0,0 +1,93 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.error.DruidException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class IsBooleanDimFilterTest extends InitializedNullHandlingTest +{ + @Test + public void testSerde() throws JsonProcessingException + { + ObjectMapper mapper = new DefaultObjectMapper(); + EqualityFilter baseFilter = new EqualityFilter("x", ColumnType.STRING, "hello", null); + + IsTrueDimFilter trueFilter = IsTrueDimFilter.of(baseFilter); + String s = mapper.writeValueAsString(trueFilter); + Assert.assertEquals(trueFilter, mapper.readValue(s, IsTrueDimFilter.class)); + + IsFalseDimFilter falseFilter = IsFalseDimFilter.of(baseFilter); + s = mapper.writeValueAsString(falseFilter); + Assert.assertEquals(falseFilter, mapper.readValue(s, IsFalseDimFilter.class)); + + } + + @Test + public void testGetCacheKey() + { + EqualityFilter f1 = new EqualityFilter("x", ColumnType.STRING, "hello", null); + EqualityFilter f1_2 = new EqualityFilter("x", ColumnType.STRING, "hello", null); + EqualityFilter f2 = new EqualityFilter("x", ColumnType.STRING, "world", null); + EqualityFilter f3 = new EqualityFilter("x", ColumnType.STRING, "hello", new FilterTuning(true, null, null)); + Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey()); + Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); + Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); + + } + + @Test + public void testInvalidParameters() + { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new IsTrueDimFilter(null) + ); + Assert.assertEquals("IS TRUE operator requires a non-null filter for field", t.getMessage()); + t = Assert.assertThrows( + DruidException.class, + () -> new IsFalseDimFilter(null) + ); + Assert.assertEquals("IS FALSE operator requires a non-null filter for field", t.getMessage()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(IsTrueDimFilter.class).usingGetClass() + .withNonnullFields("field") + .withIgnoredFields("cachedOptimizedFilter") + .verify(); + + EqualsVerifier.forClass(IsFalseDimFilter.class).usingGetClass() + .withNonnullFields("field") + .withIgnoredFields("cachedOptimizedFilter") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java index d0a7d94c3967..62caef2493a2 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java @@ -40,6 +40,8 @@ import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.StringFormatExtractionFn; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.TestHelper; @@ -704,6 +706,228 @@ private Iterable runQuery(final GroupByQuery query, final Incremental return GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query); } + @Test + public void testGroupByOnUnnestedFilterMatch() + { + // testGroupByOnUnnestedColumn but with filter to match single value + cannotVectorize(); + + final DataSource unnestDataSource = UnnestDataSource.create( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new ExpressionVirtualColumn( + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, + "\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"", + null, + ExprMacroTable.nil() + ), + null + ); + + GroupByQuery query = makeQueryBuilder() + .setDataSource(unnestDataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions( + new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0") + ) + .setDimFilter( + new EqualityFilter(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, ColumnType.STRING, "a", null) + ) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setGranularity(QueryRunnerTestHelper.ALL_GRAN) + .addOrderByColumn("alias0", OrderByColumnSpec.Direction.ASCENDING) + .build(); + + List expectedResults = Collections.singletonList( + makeRow( + query, + "2011-04-01", + "alias0", "a", + "rows", 2L + ) + ); + + Iterable results = runQuery(query, TestIndex.getIncrementalTestIndex()); + TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-virtual-column"); + } + + @Test + public void testGroupByOnUnnestedNotFilterMatch() + { + // testGroupByOnUnnestedColumn but with negated filter to match everything except 1 value + cannotVectorize(); + + final DataSource unnestDataSource = UnnestDataSource.create( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new ExpressionVirtualColumn( + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, + "\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"", + null, + ExprMacroTable.nil() + ), + null + ); + + GroupByQuery query = makeQueryBuilder() + .setDataSource(unnestDataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions( + new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0") + ) + .setDimFilter( + NotDimFilter.of(new EqualityFilter(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, ColumnType.STRING, "a", null)) + ) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setGranularity(QueryRunnerTestHelper.ALL_GRAN) + .addOrderByColumn("alias0", OrderByColumnSpec.Direction.ASCENDING) + .build(); + + List expectedResults = Arrays.asList( + makeRow( + query, + "2011-04-01", + "alias0", "b", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "e", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "h", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "m", + "rows", 6L + ), + makeRow( + query, + "2011-04-01", + "alias0", "n", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "p", + "rows", 6L + ), + makeRow( + query, + "2011-04-01", + "alias0", "preferred", + "rows", 26L + ), + makeRow( + query, + "2011-04-01", + "alias0", "t", + "rows", 4L + ) + ); + + Iterable results = runQuery(query, TestIndex.getIncrementalTestIndex()); + TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-virtual-column"); + } + + @Test + public void testGroupByOnUnnestedNotFilterMatchNonexistentValue() + { + // testGroupByOnUnnestedColumn but with negated filter on nonexistent value to still match everything + cannotVectorize(); + + final DataSource unnestDataSource = UnnestDataSource.create( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new ExpressionVirtualColumn( + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, + "\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"", + null, + ExprMacroTable.nil() + ), + null + ); + + GroupByQuery query = makeQueryBuilder() + .setDataSource(unnestDataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions( + new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0") + ) + .setDimFilter( + NotDimFilter.of(new EqualityFilter(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, ColumnType.STRING, "noexist", null)) + ) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setGranularity(QueryRunnerTestHelper.ALL_GRAN) + .addOrderByColumn("alias0", OrderByColumnSpec.Direction.ASCENDING) + .build(); + + List expectedResults = Arrays.asList( + makeRow( + query, + "2011-04-01", + "alias0", "a", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "b", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "e", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "h", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "m", + "rows", 6L + ), + makeRow( + query, + "2011-04-01", + "alias0", "n", + "rows", 2L + ), + makeRow( + query, + "2011-04-01", + "alias0", "p", + "rows", 6L + ), + makeRow( + query, + "2011-04-01", + "alias0", "preferred", + "rows", 26L + ), + makeRow( + query, + "2011-04-01", + "alias0", "t", + "rows", 4L + ) + ); + + Iterable results = runQuery(query, TestIndex.getIncrementalTestIndex()); + TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-virtual-column"); + } + private Map makeContext() { return ImmutableMap.builder() diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 286a636e89a3..6df1766bf984 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; @@ -300,12 +301,6 @@ public void test_two_levels_of_unnest_adapters() }); } - private static void assertColumnReadsIdentifier(final VirtualColumn column, final String identifier) - { - MatcherAssert.assertThat(column, CoreMatchers.instanceOf(ExpressionVirtualColumn.class)); - Assert.assertEquals("\"" + identifier + "\"", ((ExpressionVirtualColumn) column).getExpression()); - } - @Test public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest_adapters() { @@ -350,6 +345,7 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest return null; }); } + @Test public void test_nested_filters_unnested_and_original_dimension_with_unnest_adapters() { @@ -483,7 +479,6 @@ public void test_nested_filters_unnested_and_topLevelORAnd3filtersInOR() "(unnested-multi-string1 = 3 || (newcol = 2 && multi-string1 = 2 && unnested-multi-string1 = 1))" ); } - @Test public void test_nested_filters_unnested_and_topLevelAND3filtersInORWithNestedOrs() { @@ -734,6 +729,62 @@ public void test_pushdown_filters_unnested_dimension_outside() }); } + @Test + public void testUnnestValueMatcherValueDoesntExist() + { + final String inputColumn = "multi-string5"; + final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("expression-testbench"); + + final DataSegment dataSegment = DataSegment.builder() + .dataSource("foo") + .interval(schemaInfo.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + final SegmentGenerator segmentGenerator = CLOSER.register(new SegmentGenerator()); + + IncrementalIndex index = CLOSER.register( + segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, 100) + ); + IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + UnnestStorageAdapter withNullsStorageAdapter = new UnnestStorageAdapter( + adapter, + new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), + null + ); + Sequence cursorSequence = withNullsStorageAdapter.makeCursors( + null, + withNullsStorageAdapter.getInterval(), + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ); + + cursorSequence.accumulate(null, (accumulated, cursor) -> { + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); + // wont match anything + ValueMatcher matcher = dimSelector.makeValueMatcher("x"); + int count = 0; + while (!cursor.isDone()) { + Object dimSelectorVal = dimSelector.getObject(); + if (dimSelectorVal == null) { + Assert.assertNull(dimSelectorVal); + Assert.assertTrue(matcher.matches(true)); + } + Assert.assertFalse(matcher.matches(false)); + cursor.advance(); + count++; + } + Assert.assertEquals(count, 618); + return null; + }); + + } + public void testComputeBaseAndPostUnnestFilters( Filter testQueryFilter, String expectedBasePushDown, @@ -777,6 +828,12 @@ public void testComputeBaseAndPostUnnestFilters( actualPostUnnestFilter == null ? "" : actualPostUnnestFilter.toString() ); } + + private static void assertColumnReadsIdentifier(final VirtualColumn column, final String identifier) + { + MatcherAssert.assertThat(column, CoreMatchers.instanceOf(ExpressionVirtualColumn.class)); + Assert.assertEquals("\"" + identifier + "\"", ((ExpressionVirtualColumn) column).getExpression()); + } } /** diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index 5403ac7aca6c..06ca5ab3fd48 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -37,6 +37,8 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.FilterTuning; +import org.apache.druid.query.filter.IsFalseDimFilter; +import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; @@ -255,6 +257,26 @@ public void testSingleValueStringColumnWithNulls() NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), ImmutableList.of("0", "2", "4") ); + // "(s0 = 'a') is not true", same rows as "s0 <> 'a'", but also with null rows + assertFilterMatches( + NotDimFilter.of(IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))), + ImmutableList.of("0", "2", "3", "4") + ); + // "(s0 = 'a') is true", equivalent to "s0 = 'a'" + assertFilterMatches( + IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("1", "5") + ); + // "(s0 = 'a') is false", equivalent results to "s0 <> 'a'" + assertFilterMatches( + IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "4") + ); + // "(s0 = 'a') is not false", same rows as "s0 = 'a'", but also with null rows + assertFilterMatches( + NotDimFilter.of(IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))), + ImmutableList.of("1", "3", "5") + ); try { // make sure if 3vl is disabled with behave with 2vl @@ -288,6 +310,28 @@ public void testSingleValueStringColumnWithNulls() NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "noexist", null)), ImmutableList.of("0", "1", "2", "3", "4", "5") ); + + // in default value mode, is true/is false are basically pointless since they have the same behavior as = and <> + // "(s0 = 'a') is not true" equivalent to "s0 <> 'a'" + assertFilterMatches( + NotDimFilter.of(IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))), + ImmutableList.of("0", "2", "3", "4") + ); + // "(s0 = 'a') is true", equivalent to "s0 = 'a'" + assertFilterMatches( + IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("1", "5") + ); + // "(s0 = 'a') is false" equivalent to "s0 <> 'a'" + assertFilterMatches( + IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "3", "4") + ); + // "(s0 = 'a') is not false", equivalent to "s0 = 'a'" + assertFilterMatches( + NotDimFilter.of(IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))), + ImmutableList.of("1", "5") + ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/transform/TransformerTest.java b/processing/src/test/java/org/apache/druid/segment/transform/TransformerTest.java index c1aa1d0c6e88..0c33b3c2d1db 100644 --- a/processing/src/test/java/org/apache/druid/segment/transform/TransformerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/transform/TransformerTest.java @@ -389,7 +389,7 @@ public void testTransformWithArrayLongInputs() Assert.assertNotNull(actual); Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions()); Assert.assertArrayEquals(new Object[]{1L, 2L, null, 3L}, (Object[]) actual.getRaw("dim")); - Assert.assertArrayEquals(new String[]{"1", "2", null, "3"}, actual.getDimension("dim").toArray()); + Assert.assertEquals(ImmutableList.of("1", "2", "null", "3"), actual.getDimension("dim")); Assert.assertEquals(row.getTimestamp(), actual.getTimestamp()); } @@ -416,9 +416,9 @@ public void testTransformWithArrayFloatInputs() Assert.assertEquals(2.3, (Double) raw[1], 0.00001); Assert.assertNull(raw[2]); Assert.assertEquals(3.4, (Double) raw[3], 0.00001); - Assert.assertArrayEquals( - new String[]{"1.2000000476837158", "2.299999952316284", null, "3.4000000953674316"}, - actual.getDimension("dim").toArray() + Assert.assertEquals( + ImmutableList.of("1.2000000476837158", "2.299999952316284", "null", "3.4000000953674316"), + actual.getDimension("dim") ); Assert.assertEquals(row.getTimestamp(), actual.getTimestamp()); } @@ -445,12 +445,12 @@ public void testTransformWithArrayDoubleInputs() Assert.assertEquals(2.3, (Double) raw[1], 0.0); Assert.assertNull(raw[2]); Assert.assertEquals(3.4, (Double) raw[3], 0.0); - Assert.assertArrayEquals(new String[]{"1.2", "2.3", null, "3.4"}, actual.getDimension("dim").toArray()); + Assert.assertEquals(ImmutableList.of("1.2", "2.3", "null", "3.4"), actual.getDimension("dim")); Assert.assertEquals(row.getTimestamp(), actual.getTimestamp()); } @Test - public void testTransformWithExpr() + public void testTransformWithArrayExpr() { final Transformer transformer = new Transformer( new TransformSpec( @@ -517,6 +517,6 @@ public int compareTo(Row o) }); Assert.assertEquals(actualTranformedRow.getDimension("dim"), dimList.subList(0, 5)); Assert.assertArrayEquals(dimList.subList(0, 5).toArray(), (Object[]) actualTranformedRow.getRaw("dim")); - Assert.assertArrayEquals(new Object[]{"a"}, actualTranformedRow.getDimension("dim1").toArray()); + Assert.assertEquals(ImmutableList.of("a"), actualTranformedRow.getDimension("dim1")); } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 65f1087a1215..0e41f33d85e0 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -82,8 +82,9 @@ default Collection retrieveUsedSegmentsForInterval( Collection retrieveAllUsedSegments(String dataSource, Segments visibility); /** + * * Retrieve all published segments which are marked as used and the created_date of these segments belonging to the - * given data source from the metadata store. + * given data source and interval from the metadata store. * * Unlike other similar methods in this interface, this method doesn't accept a {@link Segments} "visibility" * parameter. The returned collection may include overshadowed segments and their created_dates, as if {@link @@ -91,10 +92,11 @@ default Collection retrieveUsedSegmentsForInterval( * if needed. * * @param dataSource The data source to query + * @param interval The interval to query * * @return The DataSegments and the related created_date of segments */ - Collection> retrieveUsedSegmentsAndCreatedDates(String dataSource); + Collection> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval); /** * Retrieve all published segments which may include any data in the given intervals and are marked as used from the diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 0eb96ae1061d..a58722bef8ca 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -174,15 +174,34 @@ private Collection doRetrieveUsedSegments( } @Override - public List> retrieveUsedSegmentsAndCreatedDates(String dataSource) + public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval) { - String rawQueryString = "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true"; - final String queryString = StringUtils.format(rawQueryString, dbTables.getSegmentsTable()); + StringBuilder queryBuilder = new StringBuilder( + "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true" + ); + + final List intervals = new ArrayList<>(); + // Do not need an interval condition if the interval is ETERNITY + if (!Intervals.isEternity(interval)) { + intervals.add(interval); + } + + SqlSegmentsMetadataQuery.appendConditionForIntervalsAndMatchMode( + queryBuilder, + intervals, + SqlSegmentsMetadataQuery.IntervalMode.OVERLAPS, + connector + ); + + final String queryString = StringUtils.format(queryBuilder.toString(), dbTables.getSegmentsTable()); return connector.retryWithHandle( handle -> { Query> query = handle .createQuery(queryString) .bind("dataSource", dataSource); + + SqlSegmentsMetadataQuery.bindQueryIntervals(query, intervals); + return query .map((int index, ResultSet r, StatementContext ctx) -> new Pair<>( diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java index 20b176c50914..76e4f9745762 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -261,6 +261,82 @@ public DataSegment retrieveSegmentForId(String id) return null; } + /** + * Append the condition for the interval and match mode to the given string builder with a partial query + * @param sb - StringBuilder containing the paritial query with SELECT clause and WHERE condition for used, datasource + * @param intervals - intervals to fetch the segments for + * @param matchMode - Interval match mode - overlaps or contains + * @param connector - SQL connector + */ + public static void appendConditionForIntervalsAndMatchMode( + final StringBuilder sb, + final Collection intervals, + final IntervalMode matchMode, + final SQLMetadataConnector connector + ) + { + if (intervals.isEmpty()) { + return; + } + + sb.append(" AND ("); + for (int i = 0; i < intervals.size(); i++) { + sb.append( + matchMode.makeSqlCondition( + connector.getQuoteString(), + StringUtils.format(":start%d", i), + StringUtils.format(":end%d", i) + ) + ); + + // Add a special check for a segment which have one end at eternity and the other at some finite value. Since + // we are using string comparison, a segment with this start or end will not be returned otherwise. + if (matchMode.equals(IntervalMode.OVERLAPS)) { + sb.append(StringUtils.format( + " OR (start = '%s' AND \"end\" != '%s' AND \"end\" > :start%d)", + Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i + )); + sb.append(StringUtils.format( + " OR (start != '%s' AND \"end\" = '%s' AND start < :end%d)", + Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i + )); + } + + if (i != intervals.size() - 1) { + sb.append(" OR "); + } + } + + // Add a special check for a single segment with eternity. Since we are using string comparison, a segment with + // this start and end will not be returned otherwise. + // Known Issue: https://github.com/apache/druid/issues/12860 + if (matchMode.equals(IntervalMode.OVERLAPS)) { + sb.append(StringUtils.format( + " OR (start = '%s' AND \"end\" = '%s')", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd() + )); + } + sb.append(")"); + } + + /** + * Given a Query object bind the input intervals to it + * @param query Query to fetch segments + * @param intervals Intervals to fetch segments for + */ + public static void bindQueryIntervals(final Query> query, final Collection intervals) + { + if (intervals.isEmpty()) { + return; + } + + final Iterator iterator = intervals.iterator(); + for (int i = 0; iterator.hasNext(); i++) { + Interval interval = iterator.next(); + query.bind(StringUtils.format("start%d", i), interval.getStart().toString()) + .bind(StringUtils.format("end%d", i), interval.getEnd().toString()); + } + } + private CloseableIterator retrieveSegments( final String dataSource, final Collection intervals, @@ -275,36 +351,8 @@ private CloseableIterator retrieveSegments( final StringBuilder sb = new StringBuilder(); sb.append("SELECT payload FROM %s WHERE used = :used AND dataSource = :dataSource"); - if (compareAsString && !intervals.isEmpty()) { - sb.append(" AND ("); - for (int i = 0; i < intervals.size(); i++) { - sb.append( - matchMode.makeSqlCondition( - connector.getQuoteString(), - StringUtils.format(":start%d", i), - StringUtils.format(":end%d", i) - ) - ); - - // Add a special check for a segment which have one end at eternity and the other at some finite value. Since - // we are using string comparison, a segment with this start or end will not be returned otherwise. - if (matchMode.equals(IntervalMode.OVERLAPS)) { - sb.append(StringUtils.format(" OR (start = '%s' AND \"end\" != '%s' AND \"end\" > :start%d)", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i)); - sb.append(StringUtils.format(" OR (start != '%s' AND \"end\" = '%s' AND start < :end%d)", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i)); - } - - if (i != intervals.size() - 1) { - sb.append(" OR "); - } - } - - // Add a special check for a single segment with eternity. Since we are using string comparison, a segment with - // this start and end will not be returned otherwise. - // Known Issue: https://github.com/apache/druid/issues/12860 - if (matchMode.equals(IntervalMode.OVERLAPS)) { - sb.append(StringUtils.format(" OR (start = '%s' AND \"end\" = '%s')", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd())); - } - sb.append(")"); + if (compareAsString) { + appendConditionForIntervalsAndMatchMode(sb, intervals, matchMode, connector); } final Query> sql = handle @@ -317,12 +365,7 @@ private CloseableIterator retrieveSegments( } if (compareAsString) { - final Iterator iterator = intervals.iterator(); - for (int i = 0; iterator.hasNext(); i++) { - Interval interval = iterator.next(); - sql.bind(StringUtils.format("start%d", i), interval.getStart().toString()) - .bind(StringUtils.format("end%d", i), interval.getEnd().toString()); - } + bindQueryIntervals(sql, intervals); } final ResultIterator resultIterator = diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 5c416b561939..a8fc9e923c5c 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -2652,6 +2652,52 @@ public void testGetPendingSegmentsForIntervalWithSequencePrefixes() Assert.assertEquals(expected, actual); } + @Test + public void testRetrieveUsedSegmentsAndCreatedDates() + { + insertUsedSegments(ImmutableSet.of(defaultSegment)); + + List> resultForIntervalOnTheLeft = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2000/2001")); + Assert.assertTrue(resultForIntervalOnTheLeft.isEmpty()); + + List> resultForIntervalOnTheRight = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("3000/3001")); + Assert.assertTrue(resultForIntervalOnTheRight.isEmpty()); + + List> resultForExactInterval = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), defaultSegment.getInterval()); + Assert.assertEquals(1, resultForExactInterval.size()); + Assert.assertEquals(defaultSegment, resultForExactInterval.get(0).lhs); + + List> resultForIntervalWithLeftOverlap = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2000/2015-01-02")); + Assert.assertEquals(resultForExactInterval, resultForIntervalWithLeftOverlap); + + List> resultForIntervalWithRightOverlap = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2015-01-01/3000")); + Assert.assertEquals(resultForExactInterval, resultForIntervalWithRightOverlap); + + List> resultForEternity = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.ETERNITY); + Assert.assertEquals(resultForExactInterval, resultForEternity); + } + + @Test + public void testRetrieveUsedSegmentsAndCreatedDatesFetchesEternityForAnyInterval() + { + + insertUsedSegments(ImmutableSet.of(eternitySegment, firstHalfEternityRangeSegment, secondHalfEternityRangeSegment)); + + List> resultForRandomInterval = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), defaultSegment.getInterval()); + Assert.assertEquals(3, resultForRandomInterval.size()); + + List> resultForEternity = + coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), eternitySegment.getInterval()); + Assert.assertEquals(3, resultForEternity.size()); + } + private static class DS { static final String WIKI = "wiki"; diff --git a/services/src/main/java/org/apache/druid/server/AsyncQueryForwardingServlet.java b/services/src/main/java/org/apache/druid/server/AsyncQueryForwardingServlet.java index 75b13a39f1ff..5134a8109b8a 100644 --- a/services/src/main/java/org/apache/druid/server/AsyncQueryForwardingServlet.java +++ b/services/src/main/java/org/apache/druid/server/AsyncQueryForwardingServlet.java @@ -56,6 +56,7 @@ import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.Authenticator; import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.sql.http.SqlResource; import org.eclipse.jetty.client.HttpClient; @@ -303,6 +304,7 @@ protected void service(HttpServletRequest request, HttpServletResponse response) /** * Rebuilds the {@link SqlQuery} object with sqlQueryId and queryId context parameters if not present + * * @param sqlQuery the original SqlQuery * @return an updated sqlQuery object with sqlQueryId and queryId context parameters */ @@ -367,13 +369,16 @@ void handleQueryParseException( // Log the error message final String errorMessage = exceptionToReport.getMessage() == null ? "no error message" : exceptionToReport.getMessage(); + + AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(request); + if (isNativeQuery) { requestLogger.logNativeQuery( RequestLogLine.forNative( null, DateTimes.nowUtc(), request.getRemoteAddr(), - new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) + new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage, "identity", authenticationResult.getIdentity())) ) ); } else { @@ -383,7 +388,7 @@ void handleQueryParseException( null, DateTimes.nowUtc(), request.getRemoteAddr(), - new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) + new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage, "identity", authenticationResult.getIdentity())) ) ); } @@ -744,6 +749,8 @@ public void onComplete(Result result) } emitQueryTime(requestTimeNs, success, sqlQueryId, queryId); + AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + //noinspection VariableNotUsedInsideIf if (sqlQueryId != null) { // SQL query doesn't have a native query translation in router. Hence, not logging the native query. @@ -761,7 +768,9 @@ public void onComplete(Result result) TimeUnit.NANOSECONDS.toMillis(requestTimeNs), "success", success - && result.getResponse().getStatus() == Status.OK.getStatusCode() + && result.getResponse().getStatus() == Status.OK.getStatusCode(), + "identity", + authenticationResult.getIdentity() ) ) ) @@ -787,7 +796,9 @@ public void onComplete(Result result) TimeUnit.NANOSECONDS.toMillis(requestTimeNs), "success", success - && result.getResponse().getStatus() == Status.OK.getStatusCode() + && result.getResponse().getStatus() == Status.OK.getStatusCode(), + "identity", + authenticationResult.getIdentity() ) ) ) @@ -824,6 +835,7 @@ public void onFailure(Response response, Throwable failure) failedQueryCount.incrementAndGet(); emitQueryTime(requestTimeNs, false, sqlQueryId, queryId); + AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); //noinspection VariableNotUsedInsideIf if (sqlQueryId != null) { @@ -841,7 +853,9 @@ public void onFailure(Response response, Throwable failure) "success", false, "exception", - errorMessage == null ? "no message" : errorMessage + errorMessage == null ? "no message" : errorMessage, + "identity", + authenticationResult.getIdentity() ) ) ) @@ -871,7 +885,9 @@ public void onFailure(Response response, Throwable failure) "success", false, "exception", - errorMessage == null ? "no message" : errorMessage + errorMessage == null ? "no message" : errorMessage, + "identity", + authenticationResult.getIdentity() ) ) ) @@ -890,7 +906,12 @@ public void onFailure(Response response, Throwable failure) super.onFailure(response, failure); } - private void emitQueryTime(long requestTimeNs, boolean success, @Nullable String sqlQueryId, @Nullable String queryId) + private void emitQueryTime( + long requestTimeNs, + boolean success, + @Nullable String sqlQueryId, + @Nullable String queryId + ) { QueryMetrics queryMetrics; if (sqlQueryId != null) { diff --git a/services/src/test/java/org/apache/druid/server/AsyncQueryForwardingServletTest.java b/services/src/test/java/org/apache/druid/server/AsyncQueryForwardingServletTest.java index 6facaa547780..54238fe8cce6 100644 --- a/services/src/test/java/org/apache/druid/server/AsyncQueryForwardingServletTest.java +++ b/services/src/test/java/org/apache/druid/server/AsyncQueryForwardingServletTest.java @@ -65,6 +65,8 @@ import org.apache.druid.server.router.QueryHostFinder; import org.apache.druid.server.router.RendezvousHashAvaticaConnectionBalancer; import org.apache.druid.server.security.AllowAllAuthorizer; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthenticatorMapper; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; @@ -227,7 +229,7 @@ public void testSqlQueryProxy() throws Exception Properties properties = new Properties(); properties.setProperty("druid.router.sql.enable", "true"); - verifyServletCallsForQuery(query, true, false, hostFinder, properties); + verifyServletCallsForQuery(query, true, false, hostFinder, properties, false); } @Test @@ -244,7 +246,7 @@ public void testQueryProxy() throws Exception EasyMock.expect(hostFinder.pickServer(query)).andReturn(new TestServer("http", "1.2.3.4", 9999)).once(); EasyMock.replay(hostFinder); - verifyServletCallsForQuery(query, false, false, hostFinder, new Properties()); + verifyServletCallsForQuery(query, false, false, hostFinder, new Properties(), false); } @Test @@ -258,7 +260,7 @@ public void testJDBCSqlProxy() throws Exception .once(); EasyMock.replay(hostFinder); - verifyServletCallsForQuery(jdbcRequest, false, true, hostFinder, new Properties()); + verifyServletCallsForQuery(jdbcRequest, false, true, hostFinder, new Properties(), false); } @Test @@ -408,6 +410,7 @@ public void testHandleQueryParseExceptionWithFilterDisabled() throws Exception new Properties(), new ServerConfig() ); + Mockito.when(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).thenReturn(new AuthenticationResult("userA", "basic", "basic", null)); IOException testException = new IOException(errorMessage); servlet.handleQueryParseException(request, response, mockMapper, testException, false); ArgumentCaptor captor = ArgumentCaptor.forClass(Exception.class); @@ -454,6 +457,7 @@ public ErrorResponseTransformStrategy getErrorResponseTransformStrategy() } } ); + Mockito.when(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).thenReturn(new AuthenticationResult("userA", "basic", "basic", null)); IOException testException = new IOException(errorMessage); servlet.handleQueryParseException(request, response, mockMapper, testException, false); ArgumentCaptor captor = ArgumentCaptor.forClass(Exception.class); @@ -501,6 +505,7 @@ public ErrorResponseTransformStrategy getErrorResponseTransformStrategy() } } ); + Mockito.when(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).thenReturn(new AuthenticationResult("userA", "basic", "basic", null)); IOException testException = new IOException(errorMessage); servlet.handleQueryParseException(request, response, mockMapper, testException, false); ArgumentCaptor captor = ArgumentCaptor.forClass(Exception.class); @@ -512,6 +517,46 @@ public ErrorResponseTransformStrategy getErrorResponseTransformStrategy() Assert.assertNull(((QueryException) captor.getValue()).getHost()); } + @Test + public void testNativeQueryProxyFailure() throws Exception + { + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals("2000/P1D") + .granularity(Granularities.ALL) + .context(ImmutableMap.of("queryId", "dummy")) + .build(); + + final QueryHostFinder hostFinder = EasyMock.createMock(QueryHostFinder.class); + EasyMock.expect(hostFinder.pickServer(query)).andReturn(new TestServer("http", "1.2.3.4", 9999)).once(); + EasyMock.replay(hostFinder); + + verifyServletCallsForQuery(query, false, false, hostFinder, new Properties(), true); + } + + @Test + public void testSqlQueryProxyFailure() throws Exception + { + final SqlQuery query = new SqlQuery( + "SELECT * FROM foo", + ResultFormat.ARRAY, + false, + false, + false, + ImmutableMap.of("sqlQueryId", "dummy"), + null + ); + final QueryHostFinder hostFinder = EasyMock.createMock(QueryHostFinder.class); + EasyMock.expect(hostFinder.findServerSql( + query.withOverridenContext(ImmutableMap.of("sqlQueryId", "dummy", "queryId", "dummy"))) + ).andReturn(new TestServer("http", "1.2.3.4", 9999)).once(); + EasyMock.replay(hostFinder); + + Properties properties = new Properties(); + properties.setProperty("druid.router.sql.enable", "true"); + verifyServletCallsForQuery(query, true, false, hostFinder, properties, true); + } + /** * Verifies that the Servlet calls the right methods the right number of times. */ @@ -520,7 +565,8 @@ private void verifyServletCallsForQuery( boolean isNativeSql, boolean isJDBCSql, QueryHostFinder hostFinder, - Properties properties + Properties properties, + boolean isFailure ) throws Exception { final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @@ -587,27 +633,30 @@ public int read() EasyMock.expectLastCall(); requestMock.setAttribute("org.apache.druid.proxy.to.host.scheme", "http"); EasyMock.expectLastCall(); + EasyMock.expect(requestMock.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(new AuthenticationResult("userA", "basic", "basic", null)); + if (isFailure) { + EasyMock.expect(requestMock.getRemoteAddr()).andReturn("0.0.0.0:0"); + } + EasyMock.replay(requestMock); final AtomicLong didService = new AtomicLong(); final Request proxyRequestMock = Mockito.spy(Request.class); - final Result result = new Result( - proxyRequestMock, - new HttpResponse(proxyRequestMock, ImmutableList.of()) - { - @Override - public HttpFields getHeaders() - { - HttpFields httpFields = new HttpFields(); - if (isJDBCSql) { - httpFields.add(new HttpField("X-Druid-SQL-Query-Id", "jdbcDummy")); - } else if (isNativeSql) { - httpFields.add(new HttpField("X-Druid-SQL-Query-Id", "dummy")); - } - return httpFields; - } + HttpResponse response = new HttpResponse(proxyRequestMock, ImmutableList.of()) + { + @Override + public HttpFields getHeaders() + { + HttpFields httpFields = new HttpFields(); + if (isJDBCSql) { + httpFields.add(new HttpField("X-Druid-SQL-Query-Id", "jdbcDummy")); + } else if (isNativeSql) { + httpFields.add(new HttpField("X-Druid-SQL-Query-Id", "dummy")); } - ); + return httpFields; + } + }; + final Result result = new Result(proxyRequestMock, response); final StubServiceEmitter stubServiceEmitter = new StubServiceEmitter("", ""); final AsyncQueryForwardingServlet servlet = new AsyncQueryForwardingServlet( new MapQueryToolChestWarehouse(ImmutableMap.of()), @@ -640,7 +689,11 @@ protected void doService( // partial state of the servlet. Hence, only catching the exact exception to avoid possible errors. // Further, the metric assertions are also done to ensure that the metrics have emitted. try { - servlet.newProxyResponseListener(requestMock, null).onComplete(result); + if (isFailure) { + servlet.newProxyResponseListener(requestMock, null).onFailure(response, new Throwable("Proxy failed")); + } else { + servlet.newProxyResponseListener(requestMock, null).onComplete(result); + } } catch (NullPointerException ignored) { } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index 438c666227e6..7a8786e21ef5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -45,6 +45,8 @@ import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.ExpressionDimFilter; +import org.apache.druid.query.filter.IsFalseDimFilter; +import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; @@ -379,22 +381,47 @@ public static DimFilter toFilter( { final SqlKind kind = expression.getKind(); - if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) { - return toFilter( - plannerContext, - rowSignature, - virtualColumnRegistry, - Iterables.getOnlyElement(((RexCall) expression).getOperands()) - ); - } else if (kind == SqlKind.IS_FALSE || kind == SqlKind.IS_NOT_TRUE) { - return new NotDimFilter( - toFilter( + if (kind == SqlKind.IS_TRUE + || kind == SqlKind.IS_NOT_TRUE + || kind == SqlKind.IS_FALSE + || kind == SqlKind.IS_NOT_FALSE) { + if (NullHandling.useThreeValueLogic()) { + final DimFilter baseFilter = toFilter( + plannerContext, + rowSignature, + virtualColumnRegistry, + Iterables.getOnlyElement(((RexCall) expression).getOperands()) + ); + + if (kind == SqlKind.IS_TRUE) { + return IsTrueDimFilter.of(baseFilter); + } else if (kind == SqlKind.IS_NOT_TRUE) { + return NotDimFilter.of(IsTrueDimFilter.of(baseFilter)); + } else if (kind == SqlKind.IS_FALSE) { + return IsFalseDimFilter.of(baseFilter); + } else { // SqlKind.IS_NOT_FALSE + return NotDimFilter.of(IsFalseDimFilter.of(baseFilter)); + } + } else { + // legacy behavior + if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) { + return toFilter( plannerContext, rowSignature, virtualColumnRegistry, Iterables.getOnlyElement(((RexCall) expression).getOperands()) - ) - ); + ); + } else { // SqlKind.IS_FALSE || SqlKind.IS_NOT_TRUE + return new NotDimFilter( + toFilter( + plannerContext, + rowSignature, + virtualColumnRegistry, + Iterables.getOnlyElement(((RexCall) expression).getOperands()) + ) + ); + } + } } else if (kind == SqlKind.CAST && expression.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) { // Calcite sometimes leaves errant, useless cast-to-booleans inside filters. Strip them and continue. return toFilter( @@ -403,9 +430,7 @@ public static DimFilter toFilter( virtualColumnRegistry, Iterables.getOnlyElement(((RexCall) expression).getOperands()) ); - } else if (kind == SqlKind.AND - || kind == SqlKind.OR - || kind == SqlKind.NOT) { + } else if (kind == SqlKind.AND || kind == SqlKind.OR || kind == SqlKind.NOT) { final List filters = new ArrayList<>(); for (final RexNode rexNode : ((RexCall) expression).getOperands()) { final DimFilter nextFilter = toFilter( @@ -424,8 +449,7 @@ public static DimFilter toFilter( return new AndDimFilter(filters); } else if (kind == SqlKind.OR) { return new OrDimFilter(filters); - } else { - assert kind == SqlKind.NOT; + } else { // SqlKind.NOT return new NotDimFilter(Iterables.getOnlyElement(filters)); } } else { @@ -488,6 +512,11 @@ private static DimFilter toSimpleLeafFilter( final SqlKind kind = rexNode.getKind(); if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) { + if (NullHandling.useThreeValueLogic()) { + // use expression filter to get istrue or notfalse expressions for correct 3vl behavior + return toExpressionLeafFilter(plannerContext, rowSignature, rexNode); + } + // legacy behavior return toSimpleLeafFilter( plannerContext, rowSignature, @@ -495,6 +524,11 @@ private static DimFilter toSimpleLeafFilter( Iterables.getOnlyElement(((RexCall) rexNode).getOperands()) ); } else if (kind == SqlKind.IS_FALSE || kind == SqlKind.IS_NOT_TRUE) { + if (NullHandling.useThreeValueLogic()) { + // use expression filter to get isfalse or nottrue expressions for correct 3vl behavior + return toExpressionLeafFilter(plannerContext, rowSignature, rexNode); + } + // legacy behavior return new NotDimFilter( toSimpleLeafFilter( plannerContext, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/BottomUpTransform.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/BottomUpTransform.java index f594878b2947..a0d28f372e31 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/BottomUpTransform.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/BottomUpTransform.java @@ -23,6 +23,8 @@ import com.google.common.base.Preconditions; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.IsFalseDimFilter; +import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; @@ -89,6 +91,22 @@ private DimFilter apply0(final DimFilter filter) } else { return checkedProcess(filter); } + } else if (filter instanceof IsTrueDimFilter) { + final DimFilter oldFilter = ((IsTrueDimFilter) filter).getField(); + final DimFilter newFilter = apply0(oldFilter); + if (!oldFilter.equals(newFilter)) { + return checkedProcess(new IsTrueDimFilter(newFilter)); + } else { + return checkedProcess(filter); + } + } else if (filter instanceof IsFalseDimFilter) { + final DimFilter oldFilter = ((IsFalseDimFilter) filter).getField(); + final DimFilter newFilter = apply0(oldFilter); + if (!oldFilter.equals(newFilter)) { + return checkedProcess(new IsFalseDimFilter(newFilter)); + } else { + return checkedProcess(filter); + } } else { return checkedProcess(filter); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Filtration.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Filtration.java index df03ff9662be..95466635a114 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Filtration.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Filtration.java @@ -114,6 +114,7 @@ public Filtration optimize(final RowSignature rowSignature) MoveTimeFiltersToIntervals.instance(), ConvertBoundsToSelectors.create(rowSignature), ConvertSelectorsToIns.create(rowSignature), + RemoveRedundantIsTrue.instance(), MoveMarkerFiltersToIntervals.instance(), ValidateNoMarkerFiltersRemain.instance() ) @@ -136,7 +137,8 @@ public Filtration optimizeFilterOnly(final RowSignature rowSignature) ImmutableList.of( CombineAndSimplifyBounds.instance(), ConvertBoundsToSelectors.create(rowSignature), - ConvertSelectorsToIns.create(rowSignature) + ConvertSelectorsToIns.create(rowSignature), + RemoveRedundantIsTrue.instance() ) ); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RemoveRedundantIsTrue.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RemoveRedundantIsTrue.java new file mode 100644 index 000000000000..d21b22ef4f96 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RemoveRedundantIsTrue.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.filtration; + +import com.google.common.base.Function; +import org.apache.druid.query.filter.AndDimFilter; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.IsTrueDimFilter; +import org.apache.druid.query.filter.OrDimFilter; + +import java.util.ArrayList; +import java.util.List; + +/** + * Similar to {@link BottomUpTransform} except only removes redundant IS TRUE filters that are not inside of a NOT + * filter. The planner leaves behind stuff like `(x == y) IS TRUE` which is a pointless delegate when not living inside + * of a not filter to enforce proper three-value logic + */ +public class RemoveRedundantIsTrue implements Function +{ + private static final RemoveRedundantIsTrue INSTANCE = new RemoveRedundantIsTrue(); + + public static RemoveRedundantIsTrue instance() + { + return INSTANCE; + } + + @Override + public Filtration apply(Filtration filtration) + { + if (filtration.getDimFilter() != null) { + final Filtration retVal = Filtration.create(apply0(filtration.getDimFilter()), filtration.getIntervals()); + return filtration.equals(retVal) ? retVal : apply(retVal); + } else { + return filtration; + } + } + + private DimFilter apply0(final DimFilter filter) + { + // check for AND, OR to process their children, and unwrap any IS TRUE not living under a NOT, anything else we + // leave alone + if (filter instanceof AndDimFilter) { + final List oldFilters = ((AndDimFilter) filter).getFields(); + final List newFilters = new ArrayList<>(); + for (DimFilter oldFilter : oldFilters) { + final DimFilter newFilter = apply0(oldFilter); + if (newFilter != null) { + newFilters.add(newFilter); + } + } + if (!newFilters.equals(oldFilters)) { + return new AndDimFilter(newFilters); + } else { + return filter; + } + } else if (filter instanceof OrDimFilter) { + final List oldFilters = ((OrDimFilter) filter).getFields(); + final List newFilters = new ArrayList<>(); + for (DimFilter oldFilter : oldFilters) { + final DimFilter newFilter = apply0(oldFilter); + if (newFilter != null) { + newFilters.add(newFilter); + } + } + if (!newFilters.equals(oldFilters)) { + return new OrDimFilter(newFilters); + } else { + return filter; + } + } else if (filter instanceof IsTrueDimFilter) { + final DimFilter oldFilter = ((IsTrueDimFilter) filter).getField(); + final DimFilter newFilter = apply0(oldFilter); + if (!oldFilter.equals(newFilter)) { + return newFilter; + } else { + return oldFilter; + } + } else { + return filter; + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 9141c4db090f..29f22d46c573 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -80,7 +80,7 @@ public class PlannerContext /** * Undocumented context key, used to enable window functions. */ - public static final String CTX_ENABLE_WINDOW_FNS = "windowsAreForClosers"; + public static final String CTX_ENABLE_WINDOW_FNS = "enableWindowing"; public static final String CTX_SQL_USE_BOUNDS_AND_SELECTORS = "sqlUseBoundAndSelectors"; public static final boolean DEFAULT_SQL_USE_BOUNDS_AND_SELECTORS = NullHandling.replaceWithDefault(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index c6f41aeb2efb..1527e75fa385 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -59,6 +59,7 @@ import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; @@ -365,6 +366,11 @@ public static NotDimFilter not(DimFilter filter) return new NotDimFilter(filter); } + public static IsTrueDimFilter istrue(DimFilter filter) + { + return new IsTrueDimFilter(filter); + } + public static InDimFilter in(String dimension, Collection values, ExtractionFn extractionFn) { return new InDimFilter(dimension, values, extractionFn); 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 086633d7e59c..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 @@ -6470,7 +6470,9 @@ public void testUnnestVirtualWithColumnsAndNullIf() expressionFilter("(\"j0.unnest\" == \"m2\")"), and( isNull("j0.unnest"), - not(expressionFilter("(\"j0.unnest\" == \"m2\")")) + NullHandling.sqlCompatible() + ? not(istrue(expressionFilter("(\"j0.unnest\" == \"m2\")"))) + : not(expressionFilter("(\"j0.unnest\" == \"m2\")")) ) )) .legacy(false) 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 908ccae687b2..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 @@ -5607,7 +5607,14 @@ public void testRegressionFilteredAggregatorsSubqueryJoins(Map q or( isNull("__j0.a0"), not( - or( + NullHandling.sqlCompatible() + ? istrue( + or( + not(expressionFilter("\"__j0.d0\"")), + notNull("__j0.d0") + ) + ) + : or( not(expressionFilter("\"__j0.d0\"")), notNull("__j0.d0") ) 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 08fdbc3bfaa0..49b66f8bc9f9 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 @@ -3101,7 +3101,9 @@ public void testNullEmptyStringEquality() equality("dim2", "a", ColumnType.STRING), and( isNull("dim2"), - not(equality("dim2", "a", ColumnType.STRING)) + NullHandling.sqlCompatible() + ? not(istrue(equality("dim2", "a", ColumnType.STRING))) + : not(selector("dim2", "a")) ) ) ) @@ -3109,12 +3111,12 @@ public void testNullEmptyStringEquality() .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( NullHandling.replaceWithDefault() // Matches everything but "abc" ? new Object[]{5L} - : new Object[]{2L} + // match only null values + : new Object[]{4L} ) ); } @@ -4847,13 +4849,15 @@ public void testFilteredAggregations() ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a1", "cnt"), - not(equality("dim1", "abc", ColumnType.STRING)) + NullHandling.sqlCompatible() + ? not(istrue(equality("dim1", "abc", ColumnType.STRING))) + : not(selector("dim1", "abc")) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a2", "cnt"), - NullHandling.replaceWithDefault() - ? selector("dim1", "a", new SubstringDimExtractionFn(0, 1)) - : expressionFilter("(substring(\"dim1\", 0, 1) == 'a')") + NullHandling.sqlCompatible() + ? expressionFilter("(substring(\"dim1\", 0, 1) == 'a')") + : selector("dim1", "a", new SubstringDimExtractionFn(0, 1)) ), new FilteredAggregatorFactory( 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 969b59b5037e..04556d80795f 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 @@ -477,7 +477,6 @@ public void windowQueryTest() .skipVectorize(true) .queryContext(ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - "windowsAllTheWayDown", true, QueryContexts.ENABLE_DEBUG, true) ) .sql(testCase.getQueryString())