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

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


The following commit(s) were added to refs/heads/master by this push:
     new 34ac2f7e3bb Parallelize segment metadata queries at a data node-level 
(#18592)
34ac2f7e3bb is described below

commit 34ac2f7e3bb39ca0868afaf247bf3742f92ce6fa
Author: jtuglu1 <[email protected]>
AuthorDate: Wed Oct 1 20:42:53 2025 -0700

    Parallelize segment metadata queries at a data node-level (#18592)
    
    Switches from using a custom query runner for segmentMetadata queries to 
using ChainedExecutionQueryRunner.
    
    Currently, we use some legacy query runner to run segment metadata queries 
serially (segment-by-segment) on a data node. Although there could be some 
merit to reducing query load on the data node, this could have other 
undesirable effects on the cluster by slowing down these queries used elsewhere 
(e.g. on brokers). This noisy-neighbor concern can be prevented by using the 
already-existing query prioritization (as well as the incoming per-segment 
timeout feature) to allow users to tun [...]
    
    This also has the added benefit of removing yet another one-off case (only 
one is now GroupByMergingQueryRunner) that doesn't use 
ChainedExecutionQueryRunner, helping to unify codepaths in the native engine.
---
 .../org/apache/druid/query/ConcatQueryRunner.java  | 55 ---------------
 .../SegmentMetadataQueryRunnerFactory.java         | 78 +---------------------
 .../server/coordination/ServerManagerTest.java     |  8 ++-
 3 files changed, 8 insertions(+), 133 deletions(-)

diff --git 
a/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java 
b/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java
deleted file mode 100644
index 3647521afca..00000000000
--- a/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java
+++ /dev/null
@@ -1,55 +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.query;
-
-import com.google.common.base.Function;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.java.util.common.guava.Sequences;
-import org.apache.druid.query.context.ResponseContext;
-
-/**
-*/
-public class ConcatQueryRunner<T> implements QueryRunner<T>
-{
-  private final Sequence<QueryRunner<T>> queryRunners;
-
-  public ConcatQueryRunner(Sequence<QueryRunner<T>> queryRunners)
-  {
-    this.queryRunners = queryRunners;
-  }
-
-  @Override
-  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext 
responseContext)
-  {
-    return Sequences.concat(
-        Sequences.map(
-            queryRunners,
-            new Function<>()
-            {
-              @Override
-              public Sequence<T> apply(final QueryRunner<T> input)
-              {
-                return input.run(queryPlus, responseContext);
-              }
-            }
-        )
-    );
-  }
-}
diff --git 
a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
 
b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
index 6320df7d06c..bf7b3877e51 100644
--- 
a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
+++ 
b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
@@ -19,25 +19,17 @@
 
 package org.apache.druid.query.metadata;
 
-import com.google.common.base.Function;
-import com.google.common.util.concurrent.ListenableFuture;
 import com.google.inject.Inject;
 import org.apache.druid.data.input.impl.TimestampSpec;
-import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.query.AbstractPrioritizedQueryRunnerCallable;
-import org.apache.druid.query.ConcatQueryRunner;
-import org.apache.druid.query.Query;
-import org.apache.druid.query.QueryContext;
-import org.apache.druid.query.QueryInterruptedException;
+import org.apache.druid.query.ChainedExecutionQueryRunner;
 import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryProcessingPool;
 import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryRunnerFactory;
-import org.apache.druid.query.QueryTimeoutException;
 import org.apache.druid.query.QueryToolChest;
 import org.apache.druid.query.QueryWatcher;
 import org.apache.druid.query.aggregation.AggregatorFactory;
@@ -59,10 +51,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
 public class SegmentMetadataQueryRunnerFactory implements 
QueryRunnerFactory<SegmentAnalysis, SegmentMetadataQuery>
@@ -208,69 +196,7 @@ public class SegmentMetadataQueryRunnerFactory implements 
QueryRunnerFactory<Seg
       Iterable<QueryRunner<SegmentAnalysis>> queryRunners
   )
   {
-    return new ConcatQueryRunner<>(
-        Sequences.map(
-            Sequences.simple(queryRunners),
-            new Function<>()
-            {
-              @Override
-              public QueryRunner<SegmentAnalysis> apply(final 
QueryRunner<SegmentAnalysis> input)
-              {
-                return new QueryRunner<>()
-                {
-                  @Override
-                  public Sequence<SegmentAnalysis> run(
-                      final QueryPlus<SegmentAnalysis> queryPlus,
-                      final ResponseContext responseContext
-                  )
-                  {
-                    final Query<SegmentAnalysis> query = queryPlus.getQuery();
-                    final int priority = query.context().getPriority();
-                    final QueryPlus<SegmentAnalysis> threadSafeQueryPlus = 
queryPlus.withoutThreadUnsafeState();
-                    ListenableFuture<Sequence<SegmentAnalysis>> future = 
queryProcessingPool.submitRunnerTask(
-                        new AbstractPrioritizedQueryRunnerCallable<>(priority, 
input)
-                        {
-                          @Override
-                          public Sequence<SegmentAnalysis> call()
-                          {
-                            return 
Sequences.simple(input.run(threadSafeQueryPlus, responseContext).toList());
-                          }
-                        }
-                    );
-                    try {
-                      queryWatcher.registerQueryFuture(query, future);
-                      final QueryContext context = query.context();
-                      if (context.hasTimeout()) {
-                        return future.get(context.getTimeout(), 
TimeUnit.MILLISECONDS);
-                      } else {
-                        return future.get();
-                      }
-                    }
-                    catch (InterruptedException e) {
-                      log.warn(e, "Query interrupted, cancelling pending 
results, query id [%s]", query.getId());
-                      future.cancel(true);
-                      throw new QueryInterruptedException(e);
-                    }
-                    catch (CancellationException e) {
-                      throw new QueryInterruptedException(e);
-                    }
-                    catch (TimeoutException e) {
-                      log.info("Query timeout, cancelling pending results for 
query id [%s]", query.getId());
-                      future.cancel(true);
-                      throw new 
QueryTimeoutException(StringUtils.nonStrictFormat(
-                          "Query [%s] timed out",
-                          query.getId()
-                      ));
-                    }
-                    catch (ExecutionException e) {
-                      throw new RuntimeException(e);
-                    }
-                  }
-                };
-              }
-            }
-        )
-    );
+    return new ChainedExecutionQueryRunner<>(queryProcessingPool, 
queryWatcher, queryRunners);
   }
 
   @Override
diff --git 
a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
 
b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
index a03a229bf16..92b2dcfc8ab 100644
--- 
a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
+++ 
b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
@@ -52,7 +52,6 @@ import 
org.apache.druid.java.util.common.guava.YieldingAccumulator;
 import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
-import org.apache.druid.query.ConcatQueryRunner;
 import org.apache.druid.query.DataSource;
 import org.apache.druid.query.DefaultQueryMetrics;
 import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
@@ -790,7 +789,12 @@ public class ServerManagerTest
         Iterable<QueryRunner<Result<SearchResultValue>>> queryRunners
     )
     {
-      return new ConcatQueryRunner<>(Sequences.simple(queryRunners));
+      return (queryPlus, responseContext) -> Sequences.concat(
+          Sequences.map(
+              Sequences.simple(queryRunners),
+              runner -> runner.run(queryPlus, responseContext)
+          )
+      );
     }
 
     @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to