albertobastos commented on code in PR #14823:
URL: https://github.com/apache/pinot/pull/14823#discussion_r1936957261


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -179,6 +198,9 @@ protected abstract BrokerResponse handleRequest(long 
requestId, String query, Sq
       @Nullable HttpHeaders httpHeaders, AccessControl accessControl)
       throws Exception;
 
+  protected abstract boolean handleCancel(long queryId, int timeoutMs, 
Executor executor,

Review Comment:
   Actually the `queryId` received here always refers to a broker-generated 
internal id. The `clientQueryId -> brokerQueryId` translation is done by 
`BaseBrokerRequestHandler.cancelQueryByClientId`.
   
   Added some minimal javadoc here: 52998d3e70
   
   I tried to mimic the current code design for `handleRequest`, but it is a 
bit confusing the existance of two `handleRequest` methods here:
   
   - A public method implemented by the interface and called from the endpoint 
layer that receives a `SqlNodeAndOptions` parameter.
   - A protected method called from the previous one and already receiving a 
`requestId` and the `query`'s string itself.
   
   To increase confusion, neither of the two methods have a javadoc.
   
   This probably could get better designed if we move forward with the proposed 
_shared state_ design.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -223,4 +245,65 @@ protected static void augmentStatistics(RequestContext 
statistics, BrokerRespons
     
statistics.setExplainPlanNumMatchAllFilterSegments(response.getExplainPlanNumMatchAllFilterSegments());
     statistics.setTraceInfo(response.getTraceInfo());
   }
+
+  @Override
+  public Map<Long, String> getRunningQueries() {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    return new HashMap<>(_queriesById);
+  }
+
+  @Override
+  public boolean cancelQuery(long queryId, int timeoutMs, Executor executor, 
HttpClientConnectionManager connMgr,
+      Map<String, Integer> serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    try {
+      return handleCancel(queryId, timeoutMs, executor, connMgr, 
serverResponses);
+    } finally {
+      maybeRemoveQuery(queryId);
+    }
+  }
+
+  @Override
+  public boolean cancelQueryByClientId(String clientQueryId, int timeoutMs, 
Executor executor,
+      HttpClientConnectionManager connMgr, Map<String, Integer> 
serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    Optional<Long> requestId = _clientQueryIds.entrySet().stream()
+        .filter(e -> 
clientQueryId.equals(e.getValue())).map(Map.Entry::getKey).findFirst();
+    if (requestId.isPresent()) {
+      return cancelQuery(requestId.get(), timeoutMs, executor, connMgr, 
serverResponses);
+    } else {
+      LOGGER.warn("Query cancellation cannot be performed due to unknown 
client query id: {}", clientQueryId);
+      return false;

Review Comment:
   I just tried to be consistent with the behavior when we cancel using a 
broker-generated id: it returns `true` or `false` depending on if the query 
exists or not. If we do not found a match for the given `clientQueryId`, isn't 
that the same as saying that the query does not exist?
   
   `PinotClientRequest` is the one that receives the `false` value and decides 
to raise a `WebApplicationException` enclosing a`HTTP 500` response.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -223,4 +245,65 @@ protected static void augmentStatistics(RequestContext 
statistics, BrokerRespons
     
statistics.setExplainPlanNumMatchAllFilterSegments(response.getExplainPlanNumMatchAllFilterSegments());
     statistics.setTraceInfo(response.getTraceInfo());
   }
+
+  @Override
+  public Map<Long, String> getRunningQueries() {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    return new HashMap<>(_queriesById);
+  }
+
+  @Override
+  public boolean cancelQuery(long queryId, int timeoutMs, Executor executor, 
HttpClientConnectionManager connMgr,
+      Map<String, Integer> serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    try {
+      return handleCancel(queryId, timeoutMs, executor, connMgr, 
serverResponses);
+    } finally {
+      maybeRemoveQuery(queryId);
+    }
+  }
+
+  @Override
+  public boolean cancelQueryByClientId(String clientQueryId, int timeoutMs, 
Executor executor,
+      HttpClientConnectionManager connMgr, Map<String, Integer> 
serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    Optional<Long> requestId = _clientQueryIds.entrySet().stream()
+        .filter(e -> 
clientQueryId.equals(e.getValue())).map(Map.Entry::getKey).findFirst();
+    if (requestId.isPresent()) {
+      return cancelQuery(requestId.get(), timeoutMs, executor, connMgr, 
serverResponses);
+    } else {
+      LOGGER.warn("Query cancellation cannot be performed due to unknown 
client query id: {}", clientQueryId);
+      return false;
+    }
+  }
+
+  protected String maybeSaveQuery(long requestId, SqlNodeAndOptions 
sqlNodeAndOptions, String query) {
+    if (isQueryCancellationEnabled()) {
+      String clientRequestId = sqlNodeAndOptions.getOptions() != null
+          ? 
sqlNodeAndOptions.getOptions().get(Broker.Request.QueryOptionKey.CLIENT_QUERY_ID)
 : null;
+      _queriesById.put(requestId, query);
+      if (StringUtils.isNotBlank(clientRequestId)) {
+        _clientQueryIds.put(requestId, clientRequestId);
+        LOGGER.debug("Keep track of running query: {} (with client id {})", 
requestId, clientRequestId);
+      } else {
+        LOGGER.debug("Keep track of running query: {}", requestId);
+      }
+      return clientRequestId;
+    }
+    return null;
+  }

Review Comment:
   Agree with the responsibility separation and renaming:
   
   c201cf4c59
   
   Still not 100% convinced about the code design, though. Don't like the fact 
that both methods must be called from each subclass `handleRequest` instead of 
already forced by the base class.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -223,4 +245,65 @@ protected static void augmentStatistics(RequestContext 
statistics, BrokerRespons
     
statistics.setExplainPlanNumMatchAllFilterSegments(response.getExplainPlanNumMatchAllFilterSegments());
     statistics.setTraceInfo(response.getTraceInfo());
   }
+
+  @Override
+  public Map<Long, String> getRunningQueries() {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    return new HashMap<>(_queriesById);
+  }
+
+  @Override
+  public boolean cancelQuery(long queryId, int timeoutMs, Executor executor, 
HttpClientConnectionManager connMgr,
+      Map<String, Integer> serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    try {
+      return handleCancel(queryId, timeoutMs, executor, connMgr, 
serverResponses);
+    } finally {
+      maybeRemoveQuery(queryId);
+    }
+  }
+
+  @Override
+  public boolean cancelQueryByClientId(String clientQueryId, int timeoutMs, 
Executor executor,
+      HttpClientConnectionManager connMgr, Map<String, Integer> 
serverResponses)
+      throws Exception {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    Optional<Long> requestId = _clientQueryIds.entrySet().stream()
+        .filter(e -> 
clientQueryId.equals(e.getValue())).map(Map.Entry::getKey).findFirst();
+    if (requestId.isPresent()) {
+      return cancelQuery(requestId.get(), timeoutMs, executor, connMgr, 
serverResponses);
+    } else {
+      LOGGER.warn("Query cancellation cannot be performed due to unknown 
client query id: {}", clientQueryId);
+      return false;
+    }
+  }
+
+  protected String maybeSaveQuery(long requestId, SqlNodeAndOptions 
sqlNodeAndOptions, String query) {
+    if (isQueryCancellationEnabled()) {
+      String clientRequestId = sqlNodeAndOptions.getOptions() != null
+          ? 
sqlNodeAndOptions.getOptions().get(Broker.Request.QueryOptionKey.CLIENT_QUERY_ID)
 : null;
+      _queriesById.put(requestId, query);
+      if (StringUtils.isNotBlank(clientRequestId)) {
+        _clientQueryIds.put(requestId, clientRequestId);
+        LOGGER.debug("Keep track of running query: {} (with client id {})", 
requestId, clientRequestId);
+      } else {
+        LOGGER.debug("Keep track of running query: {}", requestId);
+      }
+      return clientRequestId;
+    }
+    return null;
+  }
+
+  protected void maybeRemoveQuery(long requestId) {

Review Comment:
   Yup, makes sense combined with the `onQueryStart` refactor.
   
   c201cf4c59



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java:
##########
@@ -810,14 +813,17 @@ protected BrokerResponse handleRequest(long requestId, 
String query, SqlNodeAndO
         //       can always list the running queries and cancel query again 
until it ends. Just that such race
         //       condition makes cancel API less reliable. This should be rare 
as it assumes sending queries out to
         //       servers takes time, but will address later if needed.
-        _queriesById.put(requestId, new QueryServers(query, 
offlineRoutingTable, realtimeRoutingTable));
-        LOGGER.debug("Keep track of running query: {}", requestId);
+        String clientRequestId = maybeSaveQuery(requestId, sqlNodeAndOptions, 
query);

Review Comment:
   Already convinced, no need for more reasons :-)



##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java:
##########
@@ -547,6 +547,17 @@ public static long now() {
     return System.currentTimeMillis();
   }
 
+  @ScalarFunction
+  public static long sleep(long millis) {

Review Comment:
   Agree with that minimal compromise so we can move forward the PR.
   
   6042ad2f14
   
   Ugly but funny trick, didn't think about it.
   
   Personally I think that if we manage to force the lazy evaluation, that 
would be enough. Without the column+constant hack, it is less painful to the 
eyes.



##########
pinot-core/src/test/java/org/apache/pinot/core/data/function/DateTimeFunctionsTest.java:
##########
@@ -778,4 +787,14 @@ public void testDateTimeConvertMultipleInvocations() {
     testMultipleInvocations(String.format("dateTimeConvert(timeCol, '%s', 
'%s', '%s')", inputFormatStr, outputFormatStr,
         outputGranularityStr), rows, expectedResults);
   }
+
+  @Test
+  public void testSleepFunction() {
+    long startTime = System.currentTimeMillis();
+    testFunction("sleep(500)", Collections.emptyList(), new GenericRow(), 
result -> {
+      assertTrue((long) result >= 500);
+    });
+    long endTime = System.currentTimeMillis();
+    assertTrue(endTime - startTime >= 500);
+  }

Review Comment:
   50 ms, final offer
   
   b458a5b286



##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java:
##########
@@ -547,6 +547,17 @@ public static long now() {
     return System.currentTimeMillis();
   }
 
+  @ScalarFunction
+  public static long sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      //TODO: handle interruption
+      //Thread.currentThread().interrupt();
+    }

Review Comment:
   Any suggestion on how we should deal with an interruption here? Just warn it 
and skip the sleep or propagate the error?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -90,6 +99,16 @@ public BaseBrokerRequestHandler(PinotConfiguration config, 
String brokerId, Brok
     _brokerTimeoutMs = config.getProperty(Broker.CONFIG_OF_BROKER_TIMEOUT_MS, 
Broker.DEFAULT_BROKER_TIMEOUT_MS);
     _queryLogger = new QueryLogger(config);
     _enableNullHandling = 
config.getProperty(Broker.CONFIG_OF_BROKER_QUERY_ENABLE_NULL_HANDLING);
+
+    boolean enableQueryCancellation =
+        
Boolean.parseBoolean(config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_ENABLE_QUERY_CANCELLATION));
+    if (enableQueryCancellation) {
+      _queriesById = new ConcurrentHashMap<>();
+      _clientQueryIds = new ConcurrentHashMap<>();
+    } else {
+      _queriesById = null;
+      _clientQueryIds = null;
+    }

Review Comment:
   I agree with the shared state refactor, we should write it down somewhere so 
we actually do it ;-)



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -74,6 +81,8 @@ public abstract class BaseBrokerRequestHandler implements 
BrokerRequestHandler {
   protected final QueryLogger _queryLogger;
   @Nullable
   protected final String _enableNullHandling;
+  protected final Map<Long, String> _queriesById;
+  protected final Map<Long, String> _clientQueryIds;

Review Comment:
   Sure e2678af9c9



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java:
##########
@@ -172,9 +173,9 @@ public 
BaseSingleStageBrokerRequestHandler(PinotConfiguration config, String bro
     }
 
     LOGGER.info("Initialized {} with broker id: {}, timeout: {}ms, query 
response limit: {}, query log max length: {}, "
-            + "query log max rate: {}, query cancellation enabled: {}", 
getClass().getSimpleName(), _brokerId,
+            + "query log max rate: {}", getClass().getSimpleName(), _brokerId,
         _brokerTimeoutMs, _queryResponseLimit, 
_queryLogger.getMaxQueryLengthToLog(), _queryLogger.getLogRateLimit(),
-        enableQueryCancellation);
+        this.isQueryCancellationEnabled());

Review Comment:
   Can't say why and how, but it was unintentional. Already restored.



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CancelQueryIntegrationTests.java:
##########
@@ -0,0 +1,222 @@
+/**
+ * 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.pinot.integration.tests;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.collect.ImmutableList;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Timer;
+import java.util.UUID;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.pinot.common.utils.ServiceStatus;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.InstanceTypeUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.apache.pinot.util.TestUtils;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.*;
+
+
+/**
+ * Integration test that checks the query cancellation feature.
+ */
+public class CancelQueryIntegrationTests extends BaseClusterIntegrationTestSet 
{
+  private static final int NUM_BROKERS = 1;
+  private static final int NUM_SERVERS = 4;
+
+  private final List<ServiceStatus.ServiceStatusCallback> 
_serviceStatusCallbacks =
+      new ArrayList<>(getNumBrokers() + getNumServers());
+
+  protected int getNumBrokers() {
+    return NUM_BROKERS;
+  }
+
+  protected int getNumServers() {
+    return NUM_SERVERS;
+  }
+
+  @Override
+  protected void overrideBrokerConf(PinotConfiguration brokerConf) {
+    super.overrideBrokerConf(brokerConf);
+    
brokerConf.setProperty(CommonConstants.Broker.CONFIG_OF_BROKER_ENABLE_QUERY_CANCELLATION,
 "true");
+  }
+
+  @Override
+  protected void overrideServerConf(PinotConfiguration serverConf) {
+    super.overrideServerConf(serverConf);
+    
serverConf.setProperty(CommonConstants.Server.CONFIG_OF_ENABLE_QUERY_CANCELLATION,
 "true");
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir);
+
+    // Start the Pinot cluster
+    startZk();
+    startController();
+    // Set hyperloglog log2m value to 12.
+    HelixConfigScope scope =
+        new 
HelixConfigScopeBuilder(HelixConfigScope.ConfigScopeProperty.CLUSTER).forCluster(getHelixClusterName())
+            .build();
+    _helixManager.getConfigAccessor()
+        .set(scope, CommonConstants.Helix.DEFAULT_HYPERLOGLOG_LOG2M_KEY, 
Integer.toString(12));
+    startBrokers(getNumBrokers());
+    startServers(getNumServers());
+
+    // Create and upload the schema and table config
+    Schema schema = createSchema();
+    addSchema(schema);
+    TableConfig tableConfig = createOfflineTableConfig();
+    addTableConfig(tableConfig);
+
+    // Unpack the Avro files
+    List<File> avroFiles = unpackAvroData(_tempDir);
+
+    // Create and upload segments. For exhaustive testing, concurrently upload 
multiple segments with the same name
+    // and validate correctness with parallel push protection enabled.
+    ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, 
schema, 0, _segmentDir, _tarDir);
+    // Create a copy of _tarDir to create multiple segments with the same name.
+    File tarDir2 = new File(_tempDir, "tarDir2");
+    FileUtils.copyDirectory(_tarDir, tarDir2);
+
+    List<File> tarDirs = new ArrayList<>();
+    tarDirs.add(_tarDir);
+    tarDirs.add(tarDir2);
+    try {
+      uploadSegments(getTableName(), TableType.OFFLINE, tarDirs);
+    } catch (Exception e) {
+      // If enableParallelPushProtection is enabled and the same segment is 
uploaded concurrently, we could get one
+      // of the three exception:
+      //   - 409 conflict of the second call enters ProcessExistingSegment
+      //   - segmentZkMetadata creation failure if both calls entered 
ProcessNewSegment
+      //   - Failed to copy segment tar file to final location due to the same 
segment pushed twice concurrently
+      // In such cases we upload all the segments again to ensure that the 
data is set up correctly.
+      assertTrue(e.getMessage().contains("Another segment upload is in 
progress for segment") || e.getMessage()
+          .contains("Failed to create ZK metadata for segment") || 
e.getMessage()
+          .contains("java.nio.file.FileAlreadyExistsException"), 
e.getMessage());
+      uploadSegments(getTableName(), _tarDir);
+    }
+
+    // Set up the H2 connection
+    setUpH2Connection(avroFiles);
+
+    // Initialize the query generator
+    setUpQueryGenerator(avroFiles);
+
+    // Set up service status callbacks
+    // NOTE: put this step after creating the table and uploading all segments 
so that brokers and servers can find the
+    // resources to monitor
+    registerCallbackHandlers();
+
+    // Wait for all documents loaded
+    waitForAllDocsLoaded(600_000L);
+  }
+
+  private void registerCallbackHandlers() {
+    List<String> instances = 
_helixAdmin.getInstancesInCluster(getHelixClusterName());
+    instances.removeIf(
+        instanceId -> !InstanceTypeUtils.isBroker(instanceId) && 
!InstanceTypeUtils.isServer(instanceId));
+    List<String> resourcesInCluster = 
_helixAdmin.getResourcesInCluster(getHelixClusterName());
+    resourcesInCluster.removeIf(resource -> 
(!TableNameBuilder.isTableResource(resource)
+        && !CommonConstants.Helix.BROKER_RESOURCE_INSTANCE.equals(resource)));
+    for (String instance : instances) {
+      List<String> resourcesToMonitor = new ArrayList<>();
+      for (String resourceName : resourcesInCluster) {
+        IdealState idealState = 
_helixAdmin.getResourceIdealState(getHelixClusterName(), resourceName);
+        for (String partitionName : idealState.getPartitionSet()) {
+          if (idealState.getInstanceSet(partitionName).contains(instance)) {
+            resourcesToMonitor.add(resourceName);
+            break;
+          }
+        }
+      }
+      _serviceStatusCallbacks.add(new 
ServiceStatus.MultipleCallbackServiceStatusCallback(ImmutableList.of(
+          new 
ServiceStatus.IdealStateAndCurrentStateMatchServiceStatusCallback(_helixManager,
 getHelixClusterName(),
+              instance, resourcesToMonitor, 100.0),
+          new 
ServiceStatus.IdealStateAndExternalViewMatchServiceStatusCallback(_helixManager,
 getHelixClusterName(),
+              instance, resourcesToMonitor, 100.0))));
+    }
+  }
+
+  @Test
+  public void testInstancesStarted() {
+    assertEquals(_serviceStatusCallbacks.size(), getNumBrokers() + 
getNumServers());
+    for (ServiceStatus.ServiceStatusCallback serviceStatusCallback : 
_serviceStatusCallbacks) {
+      assertEquals(serviceStatusCallback.getServiceStatus(), 
ServiceStatus.Status.GOOD);
+    }
+  }
+
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testCancelByClientQueryId(boolean useMultiStageQueryEngine)
+    throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+    String clientRequestId = UUID.randomUUID().toString();
+    // tricky query: use sleep with some column data to avoid Calcite from 
optimizing it on compile time
+    String sqlQuery =
+        "SET " + CommonConstants.Broker.Request.QueryOptionKey.CLIENT_QUERY_ID 
+ "='" + clientRequestId + "'; "

Review Comment:
   Ok, in case we change the const the test will break anyway.
   
   9d0f335f54



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java:
##########
@@ -415,6 +415,45 @@ public String cancelQuery(
             .build());
   }
 
+  @DELETE
+  @Path("clientQuery/{clientQueryId}")
+  @Authorize(targetType = TargetType.CLUSTER, action = 
Actions.Cluster.CANCEL_QUERY)
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Cancel a query as identified by the clientQueryId", 
notes = "No effect if no query exists for"
+      + "the given clientQueryId on the requested broker. Query may continue 
to run for a short while after calling"
+      + "cancel as it's done in a non-blocking manner. The cancel method can 
be called multiple times.")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 500, 
message = "Internal server error"),
+      @ApiResponse(code = 404, message = "Query not found on the requested 
broker")
+  })
+  public String cancelClientQuery(
+      @ApiParam(value = "ClientQueryId given by the client", required = true)
+      @PathParam("clientQueryId") String clientQueryId,
+      @ApiParam(value = "Timeout for servers to respond the cancel request") 
@QueryParam("timeoutMs")
+      @DefaultValue("3000") int timeoutMs,
+      @ApiParam(value = "Return server responses for troubleshooting") 
@QueryParam("verbose") @DefaultValue("false")
+      boolean verbose) {

Review Comment:
   We could expand the already existing endpoint by adding a `@QueryParam` to 
determine if the provided id is either internal or client-based, being internal 
as default.
   
   The only drawback here is that internal ids are `long` whereas client ids 
are `string`, so type validation could no longer been done by Jersey but by the 
method itself.
   
   
[47fb9bb786](https://github.com/apache/pinot/commit/47fb9bb786577e884b103a93cd2c23442fa6cb69)
   
   The Controller scenario is different, though. There the existing endpoint is 
`DELETE /query/{brokerId}/{queryId}`, but for clientid-based cancellations we 
do not want to know the exact broker where the query felt into, so we need an 
endpoint such as `DELETE /clientQuery/{clientQueryId}`. Can't see how to unify 
these two.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -223,4 +245,65 @@ protected static void augmentStatistics(RequestContext 
statistics, BrokerRespons
     
statistics.setExplainPlanNumMatchAllFilterSegments(response.getExplainPlanNumMatchAllFilterSegments());
     statistics.setTraceInfo(response.getTraceInfo());
   }
+
+  @Override
+  public Map<Long, String> getRunningQueries() {
+    Preconditions.checkState(isQueryCancellationEnabled(), "Query cancellation 
is not enabled on broker");
+    return new HashMap<>(_queriesById);
+  }

Review Comment:
   Yeah, makes total sense c60b9538a6



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to