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


##########
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:
   It is not something we introduced in this PR, but something I think we need 
to take care of in the future:
   
   We use BaseBrokerRequestHandler as the root/common state for the broker, 
probably for historical reasons. But that is not true. A single broker may have 
SSE, MSE, GRPC and even TSE queries running at the same time. It would be a 
better design to have a shared state between them instead of the trick we do 
with the delegate.
   
   This is something we need to improve in the future



##########
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:
   Could we use the same endpoint we already have?



##########
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:
   This method is not aligned with what I would expect by its name. This method 
does two different things I think we should split into two methods:
   1. extracts the client request id
   2. optionally associates a request id to to the client id.



##########
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:
   We need javadoc here to explain how it should work. At least we should say 
that queryId may be a client or pinot generated id.



##########
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:
   Another reason to split this method is that we may want to set 
`brokerRespose.setClientRequestId` even if query cancellation is disabled. 



##########
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:
   Did you remove the `, query cancellation enabled: {}` part on propose? It 
looks like we have an extra parameter now.



##########
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:
   I don't like the sleep trick. I mean, to have a function that wait some MS 
or until some epoch per row is great for tests. But given we didn't find a way 
to make evaluation lazy (sleep with constant arguments is executed at 
optimization phase) we had to call sleep as `sleep(col + constant)`. That is 
the trick I don't like.
   
   Instead, I suggest including an option or something like that that can be 
understood by different parts of the code so we can sleep whenever we want (in 
the broker, in the leaf operator or in SSE). 
    
   The sleep function can also be used to create attacks (see 
https://www.sqlinjection.net/time-based/). I understand the sleep function is 
not the topic of this PR but just a utility to test the PR, so I don't think it 
is correct to block the PR until we have a perfect sleep function. Therefore my 
suggestion is to at least change the implementation so this function only works 
if tests are enabled. We can do that by using this horrible Java trick: 
   
   ```java
       boolean assertEnabled = false;
       assert assertEnabled = true;
       if (assertEnabled) {
         Thread.sleep(millis);
       }
   ```



##########
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:
   nit: directly use the option name instead of the const to make it easier to 
read.



##########
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:
   In fact the second method could be called `onQueryStart` and SSE could use 
it to update _serversById



##########
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:
   In fact we can call this method something like `onQueryFinish`. The fact 
that what this class does here is to remove the query id from the different 
maps is not important for the caller.



##########
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:
   we need to fix this TODO before merging



##########
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:
   Why do we need the `maybe` prefix? Just call it `remove`. If query 
cancellation is not enabled, to remove it is a NOOP.



##########
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:
   Can we document from what to what are we mapping here?



##########
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:
   should't we throw here to notify the caller that the query id is incorrect?



##########
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:
   I think it is safer and better practice to return an immutable view of the 
map instead. In the delegate we can create a copy we can modify.



##########
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:
   nit: we can reduce time to something in the order of tens of millis



-- 
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