You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/03/08 19:41:34 UTC

[GitHub] [lucene-solr] madrob commented on a change in pull request #2403: SOLR-15164: Implement Task Management Interface

madrob commented on a change in pull request #2403:
URL: https://github.com/apache/lucene-solr/pull/2403#discussion_r589676923



##########
File path: solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
##########
@@ -117,7 +117,8 @@ public void testAllConfiguredHandlers() {
             handler.startsWith("/terms") ||
             handler.startsWith("/analysis/")||
             handler.startsWith("/debug/") ||
-            handler.startsWith("/replication")
+            handler.startsWith("/replication") ||
+                handler.startsWith("/tasks")

Review comment:
       fix indent

##########
File path: lucene/core/src/java/org/apache/lucene/search/CancellableCollector.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.lucene.index.LeafReaderContext;
+
+/** Allows a query to be cancelled */
+public class CancellableCollector implements Collector, CancellableTask {
+
+  /** Thrown when a query gets cancelled */
+  public static class QueryCancelledException extends RuntimeException {}
+
+  private Collector collector;
+  private AtomicBoolean isQueryCancelled;
+
+  public CancellableCollector(Collector collector) {
+    Objects.requireNonNull(collector, "Internal collector not provided but wrapper collector accessed");
+
+    this.collector = collector;
+    this.isQueryCancelled = new AtomicBoolean();
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+
+    if (isQueryCancelled.compareAndSet(true, false)) {

Review comment:
       Why does getLeafCollector "uncancel" it? Is this collector instance meant to be reused?

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/** List the active tasks that can be cancelled */
+public class ActiveTasksListComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "activetaskslistcomponent";
+
+    private boolean shouldProcess;
+
+    @Override
+    public void prepare(ResponseBuilder rb) throws IOException {
+        if (rb.isTaskListRequest()) {
+            shouldProcess = true;
+        }
+    }
+
+    @Override
+    public void process(ResponseBuilder rb) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        if (rb.getTaskStatusCheckUUID() != null) {
+            boolean isActiveOnThisShard = rb.req.getCore().getCancellableQueryTracker().isQueryIdActive(rb.getTaskStatusCheckUUID());
+
+            rb.rsp.add("taskStatus", isActiveOnThisShard);
+            return;
+        }
+
+        rb.rsp.add("taskList", (MapWriter) ew -> {
+            Iterator<Map.Entry<String, String>> iterator = rb.req.getCore().getCancellableQueryTracker().getActiveQueriesGenerated();
+
+            while (iterator.hasNext()) {
+                Map.Entry<String, String> entry = iterator.next();
+                ew.put(entry.getKey(), entry.getValue());
+            }
+        });
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        NamedList<String> resultList = new NamedList<>();
+
+        for (ShardResponse r : sreq.responses) {
+
+            if (rb.getTaskStatusCheckUUID() != null) {
+                boolean isTaskActiveOnShard = (boolean) r.getSolrResponse().getResponse().get("taskStatus");
+
+                if (isTaskActiveOnShard == true) {

Review comment:
       simplify

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/** List the active tasks that can be cancelled */
+public class ActiveTasksListComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "activetaskslistcomponent";
+
+    private boolean shouldProcess;
+
+    @Override
+    public void prepare(ResponseBuilder rb) throws IOException {
+        if (rb.isTaskListRequest()) {
+            shouldProcess = true;
+        }
+    }
+
+    @Override
+    public void process(ResponseBuilder rb) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        if (rb.getTaskStatusCheckUUID() != null) {
+            boolean isActiveOnThisShard = rb.req.getCore().getCancellableQueryTracker().isQueryIdActive(rb.getTaskStatusCheckUUID());
+
+            rb.rsp.add("taskStatus", isActiveOnThisShard);
+            return;
+        }
+
+        rb.rsp.add("taskList", (MapWriter) ew -> {
+            Iterator<Map.Entry<String, String>> iterator = rb.req.getCore().getCancellableQueryTracker().getActiveQueriesGenerated();
+
+            while (iterator.hasNext()) {
+                Map.Entry<String, String> entry = iterator.next();
+                ew.put(entry.getKey(), entry.getValue());
+            }
+        });
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        NamedList<String> resultList = new NamedList<>();
+
+        for (ShardResponse r : sreq.responses) {
+
+            if (rb.getTaskStatusCheckUUID() != null) {
+                boolean isTaskActiveOnShard = (boolean) r.getSolrResponse().getResponse().get("taskStatus");
+
+                if (isTaskActiveOnShard == true) {
+                    rb.rsp.getValues().add("taskStatus", rb.getTaskStatusCheckUUID() + ":" + isTaskActiveOnShard);
+                    return;
+                } else {
+                    continue;
+                }
+            }
+
+            NamedList<String> result = (NamedList<String>) r.getSolrResponse()
+                    .getResponse().get("taskList");
+
+            Iterator<Map.Entry<String, String>> iterator = result.iterator();
+
+            while (iterator.hasNext()) {
+                Map.Entry<String, String> entry = iterator.next();
+
+                resultList.add(entry.getKey(), entry.getValue());
+            }

Review comment:
       resultList.addAll(result)

##########
File path: lucene/core/src/java/org/apache/lucene/search/CancellableCollector.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.lucene.index.LeafReaderContext;
+
+/** Allows a query to be cancelled */
+public class CancellableCollector implements Collector, CancellableTask {
+
+  /** Thrown when a query gets cancelled */
+  public static class QueryCancelledException extends RuntimeException {}

Review comment:
       Should this extend CollectionTerminatedException?

##########
File path: solr/solr-ref-guide/src/task-management.adoc
##########
@@ -0,0 +1,73 @@
+= Task Management
+// 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.
+
+Solr allows users to control their running tasks by monitoring them, specifying tasks as cancellation enabled and allowing
+cancellation of the same.
+
+This is achieved using the task management interface. Currently, this is supported for queries.
+
+== Types of Operations
+Task management interface (TMI) supports the following types of operations:
+
+1. List all currently running cancellable tasks.
+2. Cancel a specific task.
+3. Query the status of a specific task.
+
+== Listing All Active Cancellable Tasks
+To list all the active cancellable tasks currently running, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/list`
+
+==== Sample Response
+
+`{responseHeader={status=0, QTime=11370}, taskList={0=q=*%3A*&canCancel=true&queryUUID=0&_stateVer_=collection1%3A4&wt=javabin&version=2, 5=q=*%3A*&canCancel=true&queryUUID=5&_stateVer_=collection1%3A4&wt=javabin&version=2, 7=q=*%3A*&canCancel=true&queryUUID=7&_stateVer_=collection1%3A4&wt=javabin&version=2}`
+
+== Cancelling An Active Cancellable Task
+To cancel an active task, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/cancel?cancelUUID=foobar`
+
+==== cancelUUID Parameter
+This parameter is used to specify the UUID of the task to be cancelled.
+
+==== Sample Response
+===== If the task UUID was found and successfully cancelled:
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 cancelled successfully}`
+
+===== If the task UUID was not found
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 not found}`
+
+===== If the cancellation failed
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 could not be cancelled successfully}`

Review comment:
       Do we get a log message in the server logs in this case? Any more detail why could not be cancelled?

##########
File path: lucene/core/src/java/org/apache/lucene/search/CancellableTask.java
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.lucene.search;
+
+/** Interface which defines a task which can be cancelled */
+public interface CancellableTask {

Review comment:
       we already have `o.a.s.client.solrj.util.Cancellable` - would be nice to tie these together.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/** List the active tasks that can be cancelled */
+public class ActiveTasksListComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "activetaskslistcomponent";
+
+    private boolean shouldProcess;
+
+    @Override
+    public void prepare(ResponseBuilder rb) throws IOException {
+        if (rb.isTaskListRequest()) {
+            shouldProcess = true;
+        }
+    }
+
+    @Override
+    public void process(ResponseBuilder rb) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        if (rb.getTaskStatusCheckUUID() != null) {
+            boolean isActiveOnThisShard = rb.req.getCore().getCancellableQueryTracker().isQueryIdActive(rb.getTaskStatusCheckUUID());
+
+            rb.rsp.add("taskStatus", isActiveOnThisShard);
+            return;
+        }
+
+        rb.rsp.add("taskList", (MapWriter) ew -> {
+            Iterator<Map.Entry<String, String>> iterator = rb.req.getCore().getCancellableQueryTracker().getActiveQueriesGenerated();
+
+            while (iterator.hasNext()) {
+                Map.Entry<String, String> entry = iterator.next();
+                ew.put(entry.getKey(), entry.getValue());
+            }
+        });
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        NamedList<String> resultList = new NamedList<>();
+
+        for (ShardResponse r : sreq.responses) {
+
+            if (rb.getTaskStatusCheckUUID() != null) {
+                boolean isTaskActiveOnShard = (boolean) r.getSolrResponse().getResponse().get("taskStatus");

Review comment:
       `getBooleanArg`

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;

Review comment:
       unused, failing precommit here.

##########
File path: solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.solr.search;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+public class TestTaskManagement extends SolrCloudTestCase {
+    private static final String COLLECTION_NAME = "collection1";
+
+    private ExecutorService executorService;
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        initCore("solrconfig.xml", "schema11.xml");
+
+        configureCluster(4)
+                .addConfig("conf", configset("sql"))
+                .configure();
+    }
+
+    @AfterClass
+    public static void tearDownCluster() throws Exception {
+        shutdownCluster();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setUp();
+
+        CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf", 2, 1)
+                .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
+                .process(cluster.getSolrClient());
+        cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
+        cluster.getSolrClient().setDefaultCollection(COLLECTION_NAME);
+
+        cluster.getSolrClient().setDefaultCollection("collection1");
+
+        executorService = ExecutorUtil.newMDCAwareCachedThreadPool("TestTaskManagement");
+
+        List<SolrInputDocument> docs = new ArrayList<>();
+        for (int i = 0; i < 100; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", i);
+            doc.addField("foo1_s", Integer.toString(i));
+            doc.addField("foo2_s", Boolean.toString(i % 2 == 0));
+            doc.addField("foo4_s", new BytesRef(Boolean.toString(i % 2 == 0)));
+
+            docs.add(doc);
+        }
+
+        cluster.getSolrClient().add(docs);
+        cluster.getSolrClient().commit();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        CollectionAdminRequest.deleteCollection(COLLECTION_NAME).process(cluster.getSolrClient());
+        executorService.shutdown();
+        super.tearDown();
+    }
+
+    @Test
+    public void testNonExistentQuery() throws Exception {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+
+        params.set("cancelUUID", "foobar");
+        @SuppressWarnings({"rawtypes"})
+        SolrRequest request = new QueryRequest(params);
+        request.setPath("/tasks/cancel");
+
+        NamedList<Object> queryResponse = null;
+
+        queryResponse = cluster.getSolrClient().request(request);
+
+        assertEquals("Query with queryID foobar not found", queryResponse.get("status"));
+    }
+
+    @Test
+    public void testCancellationQuery() throws Exception {
+        ConcurrentHashSet<Integer> queryIdsSet = new ConcurrentHashSet<>();

Review comment:
       Use `ConcurrentHashMap.newKeySet` instead

##########
File path: solr/solr-ref-guide/src/common-query-parameters.adoc
##########
@@ -84,6 +84,18 @@ You can use the `rows` parameter to paginate results from a query. The parameter
 
 The default value is `10`. That is, by default, Solr returns 10 documents at a time in response to a query.
 
+== canCancel Parameter
+
+This parameter defines if this query is cancellable i.e. can be cancelled during execution using the
+task management interface.
+
+== queryUUID Parameter
+
+For cancellable queries, this allows specifying a custom UUID to identify the query with. If `canCancel` is specified and `queryUUID` is not set, an auto generated UUID will be assigned to the query.
+
+If `queryUUID` is specified, this UUID will be used for identifying the query. Note that if using `queryUUID`, the responsibility of ensuring uniqueness of the UUID lies with the caller.

Review comment:
       What happens if a uuid is accidentally reused?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -3245,6 +3252,75 @@ public void postClose(SolrCore core) {
     return blobRef;
   }
 
+  /** Generates a UUID for the given query or if the user provided a UUID
+   * for this query, uses that.
+   */
+  public String generateQueryID(SolrQueryRequest req) {
+    String queryID;
+    String customQueryUUID = req.getParams().get(CUSTOM_QUERY_UUID, null);
+
+    if (customQueryUUID != null) {
+      queryID = customQueryUUID;
+    } else {
+      queryID = UUID.randomUUID().toString();

Review comment:
       This doesn't need to be cryptographically secure, use a different generator here.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/** List the active tasks that can be cancelled */
+public class ActiveTasksListComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "activetaskslistcomponent";
+
+    private boolean shouldProcess;
+
+    @Override
+    public void prepare(ResponseBuilder rb) throws IOException {
+        if (rb.isTaskListRequest()) {
+            shouldProcess = true;
+        }
+    }
+
+    @Override
+    public void process(ResponseBuilder rb) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        if (rb.getTaskStatusCheckUUID() != null) {
+            boolean isActiveOnThisShard = rb.req.getCore().getCancellableQueryTracker().isQueryIdActive(rb.getTaskStatusCheckUUID());
+
+            rb.rsp.add("taskStatus", isActiveOnThisShard);
+            return;
+        }
+
+        rb.rsp.add("taskList", (MapWriter) ew -> {
+            Iterator<Map.Entry<String, String>> iterator = rb.req.getCore().getCancellableQueryTracker().getActiveQueriesGenerated();
+
+            while (iterator.hasNext()) {
+                Map.Entry<String, String> entry = iterator.next();
+                ew.put(entry.getKey(), entry.getValue());
+            }
+        });
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        NamedList<String> resultList = new NamedList<>();
+
+        for (ShardResponse r : sreq.responses) {
+
+            if (rb.getTaskStatusCheckUUID() != null) {
+                boolean isTaskActiveOnShard = (boolean) r.getSolrResponse().getResponse().get("taskStatus");
+
+                if (isTaskActiveOnShard == true) {
+                    rb.rsp.getValues().add("taskStatus", rb.getTaskStatusCheckUUID() + ":" + isTaskActiveOnShard);

Review comment:
       I don't think returning UUID:true here makes semantic sense. Maybe UUID + ":active"?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestCancellableCollector.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
+
+public class TestCancellableCollector extends LuceneTestCase {
+  Directory dir;
+  IndexReader reader;
+  IndexSearcher searcher;
+  ExecutorService executor = null;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Random random = random();
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw.commit();
+      }
+    }
+    reader = iw.getReader();
+    iw.close();
+
+    searcher = new IndexSearcher(reader);
+
+    executor =
+        new ThreadPoolExecutor(
+            4,
+            4,
+            0L,
+            TimeUnit.MILLISECONDS,
+            new LinkedBlockingQueue<Runnable>(),

Review comment:
       nit, use `<>`

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestCancellableCollector.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
+
+public class TestCancellableCollector extends LuceneTestCase {
+  Directory dir;
+  IndexReader reader;
+  IndexSearcher searcher;
+  ExecutorService executor = null;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();

Review comment:
       Is a lot of this copied from LuceneTestCase? Can we reuse that code better instead?

##########
File path: solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.solr.search;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+public class TestTaskManagement extends SolrCloudTestCase {
+    private static final String COLLECTION_NAME = "collection1";
+
+    private ExecutorService executorService;
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        initCore("solrconfig.xml", "schema11.xml");
+
+        configureCluster(4)
+                .addConfig("conf", configset("sql"))
+                .configure();
+    }
+
+    @AfterClass
+    public static void tearDownCluster() throws Exception {
+        shutdownCluster();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setUp();
+
+        CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf", 2, 1)
+                .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
+                .process(cluster.getSolrClient());
+        cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
+        cluster.getSolrClient().setDefaultCollection(COLLECTION_NAME);
+
+        cluster.getSolrClient().setDefaultCollection("collection1");
+
+        executorService = ExecutorUtil.newMDCAwareCachedThreadPool("TestTaskManagement");
+
+        List<SolrInputDocument> docs = new ArrayList<>();
+        for (int i = 0; i < 100; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", i);
+            doc.addField("foo1_s", Integer.toString(i));
+            doc.addField("foo2_s", Boolean.toString(i % 2 == 0));
+            doc.addField("foo4_s", new BytesRef(Boolean.toString(i % 2 == 0)));
+
+            docs.add(doc);
+        }
+
+        cluster.getSolrClient().add(docs);
+        cluster.getSolrClient().commit();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        CollectionAdminRequest.deleteCollection(COLLECTION_NAME).process(cluster.getSolrClient());
+        executorService.shutdown();
+        super.tearDown();
+    }
+
+    @Test
+    public void testNonExistentQuery() throws Exception {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+
+        params.set("cancelUUID", "foobar");
+        @SuppressWarnings({"rawtypes"})
+        SolrRequest request = new QueryRequest(params);
+        request.setPath("/tasks/cancel");
+
+        NamedList<Object> queryResponse = null;
+
+        queryResponse = cluster.getSolrClient().request(request);
+
+        assertEquals("Query with queryID foobar not found", queryResponse.get("status"));
+    }
+
+    @Test
+    public void testCancellationQuery() throws Exception {
+        ConcurrentHashSet<Integer> queryIdsSet = new ConcurrentHashSet<>();
+        ConcurrentHashSet<Integer> notFoundIdsSet = new ConcurrentHashSet<>();
+
+        List<CompletableFuture<Void>> queryFutures = new ArrayList<>();
+
+        for (int i = 0; i < 100; i++) {
+            CompletableFuture<Void> future = executeQueryAsync(Integer.toString(i));
+
+            queryFutures.add(future);
+        }
+
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+
+        for (int i = 0; i < 90; i++) {
+            CompletableFuture<Void> future = cancelQuery(Integer.toString(i), 4000, queryIdsSet, notFoundIdsSet);
+
+            futures.add(future);
+        }
+
+        futures.forEach(CompletableFuture::join);
+
+        queryFutures.forEach(CompletableFuture::join);
+
+        assertTrue(queryIdsSet.size() + notFoundIdsSet.size() == 90);

Review comment:
       nit: assertEquals, and include a message

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestCancellableCollector.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
+
+public class TestCancellableCollector extends LuceneTestCase {
+  Directory dir;
+  IndexReader reader;
+  IndexSearcher searcher;
+  ExecutorService executor = null;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Random random = random();
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw.commit();
+      }
+    }
+    reader = iw.getReader();
+    iw.close();
+
+    searcher = new IndexSearcher(reader);
+
+    executor =
+        new ThreadPoolExecutor(
+            4,
+            4,
+            0L,
+            TimeUnit.MILLISECONDS,
+            new LinkedBlockingQueue<Runnable>(),
+            new NamedThreadFactory("TestIndexSearcher"));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    reader.close();
+    dir.close();
+
+    if (executor != null) {
+      executor.shutdown();
+    }
+
+    executor = null;
+  }
+
+  private CancellableCollector buildCancellableCollector(
+      final int numHits, boolean delayStart, boolean delayCollection) {
+    TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(numHits, null, 1);
+    CancellableCollector collector = new CancellableCollector(topScoreDocCollector);
+
+    DummyCancellableCollector dummyCancellableCollector =
+        new DummyCancellableCollector(collector, delayStart, delayCollection);
+
+    return dummyCancellableCollector;
+  }
+
+  private void executeSearchTest(
+      IndexSearcher searcher, Query query, CancellableCollector cancellableCollector, int numHits)
+      throws Exception {
+    TopDocs topDocs = searcher.search(query, numHits);
+
+    searcher.search(query, cancellableCollector);
+
+    CancellableCollector internalCancellableCollector =
+        (CancellableCollector) cancellableCollector.getInternalCollector();
+    TopScoreDocCollector topScoreDocCollector =
+        (TopScoreDocCollector) internalCancellableCollector.getInternalCollector();
+
+    assertEquals(topDocs.totalHits.value, topScoreDocCollector.totalHits);
+  }
+
+  private void cancelQuery(CancellableCollector cancellableCollector, final int sleepTime) {
+    executor.submit(
+        () -> {
+
+          // Wait for some time to let the query start
+          try {
+            if (sleepTime > 0) {
+              Thread.sleep(sleepTime);
+            }
+
+            cancellableCollector.cancelTask();
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e.getMessage());
+          }
+        });
+  }
+
+  public void testSearchWithoutCancellation() throws Exception {
+    CancellableCollector cancellableCollector = buildCancellableCollector(50, false, false);
+
+    Query query = new TermQuery(new Term("field", "1"));
+
+    executeSearchTest(searcher, query, cancellableCollector, 50);
+
+    query = new MatchAllDocsQuery();
+
+    cancellableCollector = buildCancellableCollector(100, false, false);
+
+    executeSearchTest(searcher, query, cancellableCollector, 50);
+  }
+
+  public void testSearchWithCancellationBeforeActualDocumentCollection() {
+    Query query = new MatchAllDocsQuery();
+
+    CancellableCollector cancellableCollector = buildCancellableCollector(5000, true, false);
+
+    expectThrows(
+        CancellableCollector.QueryCancelledException.class,
+        () -> {
+          // Cancel the query before the document collection starts
+          cancelQuery(cancellableCollector, 0);
+
+          executeSearchTest(searcher, query, cancellableCollector, 5000);
+        });
+  }
+
+  public void testSearchWithCancellationBetweenActualDocumentCollection() {
+    Query query = new MatchAllDocsQuery();
+
+    CancellableCollector cancellableCollector = buildCancellableCollector(5000, false, true);
+
+    expectThrows(
+        CancellableCollector.QueryCancelledException.class,
+        () -> {
+          // Cancel the query before the document collection starts
+          cancelQuery(cancellableCollector, 0);
+
+          executeSearchTest(searcher, query, cancellableCollector, 5000);
+        });
+  }
+
+  public class DummyCancellableCollector extends CancellableCollector {

Review comment:
       class should be static

##########
File path: solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.solr.search;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+public class TestTaskManagement extends SolrCloudTestCase {
+    private static final String COLLECTION_NAME = "collection1";
+
+    private ExecutorService executorService;
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        initCore("solrconfig.xml", "schema11.xml");
+
+        configureCluster(4)
+                .addConfig("conf", configset("sql"))
+                .configure();
+    }
+
+    @AfterClass
+    public static void tearDownCluster() throws Exception {
+        shutdownCluster();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setUp();
+
+        CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf", 2, 1)
+                .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
+                .process(cluster.getSolrClient());
+        cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
+        cluster.getSolrClient().setDefaultCollection(COLLECTION_NAME);
+
+        cluster.getSolrClient().setDefaultCollection("collection1");
+
+        executorService = ExecutorUtil.newMDCAwareCachedThreadPool("TestTaskManagement");
+
+        List<SolrInputDocument> docs = new ArrayList<>();
+        for (int i = 0; i < 100; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", i);
+            doc.addField("foo1_s", Integer.toString(i));
+            doc.addField("foo2_s", Boolean.toString(i % 2 == 0));
+            doc.addField("foo4_s", new BytesRef(Boolean.toString(i % 2 == 0)));
+
+            docs.add(doc);
+        }
+
+        cluster.getSolrClient().add(docs);
+        cluster.getSolrClient().commit();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        CollectionAdminRequest.deleteCollection(COLLECTION_NAME).process(cluster.getSolrClient());
+        executorService.shutdown();
+        super.tearDown();
+    }
+
+    @Test
+    public void testNonExistentQuery() throws Exception {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+
+        params.set("cancelUUID", "foobar");
+        @SuppressWarnings({"rawtypes"})
+        SolrRequest request = new QueryRequest(params);
+        request.setPath("/tasks/cancel");
+
+        NamedList<Object> queryResponse = null;
+
+        queryResponse = cluster.getSolrClient().request(request);
+
+        assertEquals("Query with queryID foobar not found", queryResponse.get("status"));
+    }
+
+    @Test
+    public void testCancellationQuery() throws Exception {
+        ConcurrentHashSet<Integer> queryIdsSet = new ConcurrentHashSet<>();
+        ConcurrentHashSet<Integer> notFoundIdsSet = new ConcurrentHashSet<>();
+
+        List<CompletableFuture<Void>> queryFutures = new ArrayList<>();
+
+        for (int i = 0; i < 100; i++) {
+            CompletableFuture<Void> future = executeQueryAsync(Integer.toString(i));
+
+            queryFutures.add(future);
+        }
+
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+
+        for (int i = 0; i < 90; i++) {
+            CompletableFuture<Void> future = cancelQuery(Integer.toString(i), 4000, queryIdsSet, notFoundIdsSet);
+
+            futures.add(future);
+        }
+
+        futures.forEach(CompletableFuture::join);
+
+        queryFutures.forEach(CompletableFuture::join);
+
+        assertTrue(queryIdsSet.size() + notFoundIdsSet.size() == 90);
+    }
+
+    @Test
+    public void testListCancellableQueries() throws Exception {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+
+        @SuppressWarnings({"rawtypes"})
+        SolrRequest request = new QueryRequest(params);
+        request.setPath("/tasks/list");
+
+        for (int i = 0; i < 50; i++) {
+            executeQueryAsync(Integer.toString(i));
+        }
+
+        NamedList<Object> queryResponse = null;
+
+        queryResponse = cluster.getSolrClient().request(request);
+
+        @SuppressWarnings({"unchecked"})
+        NamedList<String> result = (NamedList<String>) queryResponse.get("taskList");
+
+        Iterator<Map.Entry<String, String>> iterator = result.iterator();
+
+        Set<Integer> presentQueryIDs = new HashSet<>();
+
+        while (iterator.hasNext()) {
+            Map.Entry<String, String> entry = iterator.next();
+
+            presentQueryIDs.add(Integer.parseInt(entry.getKey()));
+        }
+
+        assertTrue(presentQueryIDs.size() > 0 && presentQueryIDs.size() <= 50);
+
+        Iterator<Integer> integerIterator = presentQueryIDs.iterator();
+
+        while (integerIterator.hasNext()) {
+            int value = integerIterator.next();
+
+            assertTrue (value >= 0 && value < 50);
+        }
+    }
+
+    @Test
+    public void testCheckSpecificQueryStatus() throws Exception {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+
+        params.set("taskUUID", "25");
+
+        @SuppressWarnings({"rawtypes"})
+        SolrRequest request = new QueryRequest(params);
+
+        request.setPath("/tasks/list");
+
+        NamedList<Object> queryResponse = cluster.getSolrClient().request(request);
+
+        @SuppressWarnings({"unchecked"})
+        String result = (String) queryResponse.get("taskStatus");
+
+        assertFalse(result.contains("true"));
+    }
+
+    private CompletableFuture<Void> cancelQuery(final String queryID, final int sleepTime, Set<Integer> cancelledQueryIdsSet,
+                                          Set<Integer> notFoundQueryIdSet) {
+        return CompletableFuture.runAsync(() -> {
+            ModifiableSolrParams params = new ModifiableSolrParams();
+
+            params.set("cancelUUID", queryID);
+            @SuppressWarnings({"rawtypes"})
+            SolrRequest request = new QueryRequest(params);
+            request.setPath("/tasks/cancel");
+
+            // Wait for some time to let the query start
+            try {
+                if (sleepTime > 0) {
+                    Thread.sleep(sleepTime);
+                }
+
+                try {
+                    NamedList<Object> queryResponse = null;
+
+                    queryResponse = cluster.getSolrClient().request(request);
+
+                    String cancellationResult = (String) queryResponse.get("status");
+                    if (cancellationResult.contains("cancelled successfully")) {
+                        cancelledQueryIdsSet.add(Integer.parseInt(queryID));
+                    } else if (cancellationResult.contains("not found")) {
+                        notFoundQueryIdSet.add(Integer.parseInt(queryID));
+                    }

Review comment:
       If we're having to do string comparisons like this, then that suggests the API is deficient in some way. We should return this as part of the response better. Maybe an enum or other kinds of well known values.

##########
File path: solr/solr-ref-guide/src/task-management.adoc
##########
@@ -0,0 +1,73 @@
+= Task Management
+// 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.
+
+Solr allows users to control their running tasks by monitoring them, specifying tasks as cancellation enabled and allowing
+cancellation of the same.
+
+This is achieved using the task management interface. Currently, this is supported for queries.
+
+== Types of Operations
+Task management interface (TMI) supports the following types of operations:
+
+1. List all currently running cancellable tasks.
+2. Cancel a specific task.
+3. Query the status of a specific task.
+
+== Listing All Active Cancellable Tasks
+To list all the active cancellable tasks currently running, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/list`
+
+==== Sample Response
+
+`{responseHeader={status=0, QTime=11370}, taskList={0=q=*%3A*&canCancel=true&queryUUID=0&_stateVer_=collection1%3A4&wt=javabin&version=2, 5=q=*%3A*&canCancel=true&queryUUID=5&_stateVer_=collection1%3A4&wt=javabin&version=2, 7=q=*%3A*&canCancel=true&queryUUID=7&_stateVer_=collection1%3A4&wt=javabin&version=2}`
+
+== Cancelling An Active Cancellable Task
+To cancel an active task, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/cancel?cancelUUID=foobar`
+
+==== cancelUUID Parameter
+This parameter is used to specify the UUID of the task to be cancelled.
+
+==== Sample Response
+===== If the task UUID was found and successfully cancelled:
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 cancelled successfully}`
+
+===== If the task UUID was not found
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 not found}`

Review comment:
       This is the case where the query has already completed, right?

##########
File path: solr/solr-ref-guide/src/task-management.adoc
##########
@@ -0,0 +1,73 @@
+= Task Management
+// 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.
+
+Solr allows users to control their running tasks by monitoring them, specifying tasks as cancellation enabled and allowing
+cancellation of the same.
+
+This is achieved using the task management interface. Currently, this is supported for queries.
+
+== Types of Operations
+Task management interface (TMI) supports the following types of operations:

Review comment:
       We don't need an acronym for it :)

##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -1531,6 +1567,26 @@ private void doProcessUngroupedSearch(ResponseBuilder rb, QueryCommand cmd, Quer
     doPrefetch(rb);
   }
 
+  private static String generateQueryID(SolrQueryRequest req) {
+    ZkController zkController = req.getCore().getCoreContainer().getZkController();
+    String nodeName = req.getCore().getCoreContainer().getHostName();
+
+    if (zkController != null) {
+      nodeName = req.getCore().getCoreContainer().

Review comment:
       use controller directly, don't need to chain through the core again

##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryCancellationComponent.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.lucene.search.CancellableTask;
+
+import java.io.IOException;
+
+/** Responsible for handling query cancellation requests */
+public class QueryCancellationComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "querycancellation";
+
+    private boolean shouldProcess;
+
+    @Override
+    public void prepare(ResponseBuilder rb) throws IOException
+    {
+        if (rb.isCancellation()) {
+            shouldProcess = true;
+        }
+    }
+
+    @Override
+    public void process(ResponseBuilder rb) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        String cancellationUUID = rb.getCancellationUUID();
+
+        if (cancellationUUID == null) {
+            throw new RuntimeException("Null query UUID seen");
+        }
+
+        CancellableTask cancellableTask = rb.req.getCore().getCancellableQueryTracker().getCancellableTask(cancellationUUID);
+
+        if (cancellableTask != null) {
+            cancellableTask.cancelTask();
+            rb.rsp.add("cancellationResult", "success");
+        } else {
+            rb.rsp.add("cancellationResult", "not found");
+        }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+        if (!shouldProcess) {
+            return;
+        }
+
+        boolean failureSeen = false;
+        boolean queryNotFound = false;
+
+        for (ShardResponse r : sreq.responses) {
+
+            String cancellationResult = (String) r.getSolrResponse()
+                    .getResponse().get("cancellationResult");
+
+            if (!cancellationResult.equalsIgnoreCase("success")) {
+                if (cancellationResult.equalsIgnoreCase("not found")) {
+                    queryNotFound = true;
+                } else {
+                    failureSeen = true;
+                }
+
+                break;
+            }
+        }
+
+        if (failureSeen) {
+            rb.rsp.getValues().add("status", "Query with queryID " + rb.getCancellationUUID() +
+                    " could not be cancelled successfully");
+        } else if (queryNotFound) {
+            rb.rsp.getValues().add("status", "Query with queryID " + rb.getCancellationUUID() +
+                    " not found");
+        } else {
+            rb.rsp.getValues().add("status", "Query with queryID " + rb.getCancellationUUID() +
+                    " cancelled successfully");
+        }
+    }
+
+    @Override
+    public String getDescription() {
+        return "querycancellation";

Review comment:
       return COMPONENT_NAME

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestCancellableCollector.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
+
+public class TestCancellableCollector extends LuceneTestCase {
+  Directory dir;
+  IndexReader reader;
+  IndexSearcher searcher;
+  ExecutorService executor = null;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Random random = random();
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw.commit();
+      }
+    }
+    reader = iw.getReader();
+    iw.close();
+
+    searcher = new IndexSearcher(reader);
+
+    executor =
+        new ThreadPoolExecutor(
+            4,
+            4,
+            0L,
+            TimeUnit.MILLISECONDS,
+            new LinkedBlockingQueue<Runnable>(),
+            new NamedThreadFactory("TestIndexSearcher"));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    reader.close();
+    dir.close();
+
+    if (executor != null) {
+      executor.shutdown();
+    }
+
+    executor = null;
+  }
+
+  private CancellableCollector buildCancellableCollector(
+      final int numHits, boolean delayStart, boolean delayCollection) {
+    TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(numHits, null, 1);
+    CancellableCollector collector = new CancellableCollector(topScoreDocCollector);
+
+    DummyCancellableCollector dummyCancellableCollector =
+        new DummyCancellableCollector(collector, delayStart, delayCollection);
+
+    return dummyCancellableCollector;
+  }
+
+  private void executeSearchTest(
+      IndexSearcher searcher, Query query, CancellableCollector cancellableCollector, int numHits)
+      throws Exception {
+    TopDocs topDocs = searcher.search(query, numHits);
+
+    searcher.search(query, cancellableCollector);
+
+    CancellableCollector internalCancellableCollector =
+        (CancellableCollector) cancellableCollector.getInternalCollector();
+    TopScoreDocCollector topScoreDocCollector =
+        (TopScoreDocCollector) internalCancellableCollector.getInternalCollector();
+
+    assertEquals(topDocs.totalHits.value, topScoreDocCollector.totalHits);
+  }
+
+  private void cancelQuery(CancellableCollector cancellableCollector, final int sleepTime) {
+    executor.submit(
+        () -> {
+
+          // Wait for some time to let the query start
+          try {
+            if (sleepTime > 0) {

Review comment:
       I don't understand when we want to use this. It relies on non-deterministic scheduling in the threads for the test? If we need to ensure that a task has started, have it trigger a latch that the code calling this method is waiting on.

##########
File path: solr/solr-ref-guide/src/task-management.adoc
##########
@@ -0,0 +1,73 @@
+= Task Management
+// 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.
+
+Solr allows users to control their running tasks by monitoring them, specifying tasks as cancellation enabled and allowing
+cancellation of the same.
+
+This is achieved using the task management interface. Currently, this is supported for queries.
+
+== Types of Operations
+Task management interface (TMI) supports the following types of operations:
+
+1. List all currently running cancellable tasks.
+2. Cancel a specific task.
+3. Query the status of a specific task.
+
+== Listing All Active Cancellable Tasks
+To list all the active cancellable tasks currently running, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/list`
+
+==== Sample Response
+
+`{responseHeader={status=0, QTime=11370}, taskList={0=q=*%3A*&canCancel=true&queryUUID=0&_stateVer_=collection1%3A4&wt=javabin&version=2, 5=q=*%3A*&canCancel=true&queryUUID=5&_stateVer_=collection1%3A4&wt=javabin&version=2, 7=q=*%3A*&canCancel=true&queryUUID=7&_stateVer_=collection1%3A4&wt=javabin&version=2}`
+
+== Cancelling An Active Cancellable Task
+To cancel an active task, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/cancel?cancelUUID=foobar`
+
+==== cancelUUID Parameter
+This parameter is used to specify the UUID of the task to be cancelled.
+
+==== Sample Response
+===== If the task UUID was found and successfully cancelled:
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 cancelled successfully}`
+
+===== If the task UUID was not found
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 not found}`
+
+===== If the cancellation failed
+
+`{responseHeader={status=0, QTime=39}, status=Query with queryID 85 could not be cancelled successfully}`
+
+== Check Status of a Specific Task
+To check the status of a specific task, please use the following syntax:
+
+`\http://localhost:8983/solr/tasks/list?taskUUID=foobar`
+
+==== taskUUID Parameter
+`taskUUID` parameter can be used to specify a task UUID whose status can be checked.
+
+==== Sample Response
+`{responseHeader={status=0, QTime=6128}, taskStatus=foobar:true}`

Review comment:
       this does not feel meaningful. It's just an existence check? 

##########
File path: solr/core/src/java/org/apache/solr/handler/component/ActiveTasksListComponent.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.solr.handler.component;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/** List the active tasks that can be cancelled */
+public class ActiveTasksListComponent extends SearchComponent {
+    public static final String COMPONENT_NAME = "activetaskslistcomponent";

Review comment:
       name should not end in component




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org