parent
9939e81d88
commit
91999e14ce
|
@ -70,6 +70,8 @@ include::cat/master.asciidoc[]
|
|||
|
||||
include::cat/nodes.asciidoc[]
|
||||
|
||||
include::cat/pending_tasks.asciidoc[]
|
||||
|
||||
include::cat/recovery.asciidoc[]
|
||||
|
||||
include::cat/shards.asciidoc[]
|
||||
|
|
|
@ -0,0 +1,18 @@
|
|||
[[cat-pending-tasks]]
|
||||
== Cluster Pending Tasks
|
||||
|
||||
`pending_tasks` provides the same information as the
|
||||
`/_cluster/pending_tasks` API in a convenient tabular format.
|
||||
|
||||
[source,shell]
|
||||
--------------------------------------------------
|
||||
% curl 'localhost:9200/_cat/pending_tasks?v'
|
||||
insertOrder timeInQueue priority source
|
||||
1685 855ms HIGH update-mapping [foo][t]
|
||||
1686 843ms HIGH update-mapping [foo][t]
|
||||
1693 753ms HIGH refresh-mapping [foo][[t]]
|
||||
1688 816ms HIGH update-mapping [foo][t]
|
||||
1689 802ms HIGH update-mapping [foo][t]
|
||||
1690 787ms HIGH update-mapping [foo][t]
|
||||
1691 773ms HIGH update-mapping [foo][t]
|
||||
--------------------------------------------------
|
|
@ -218,5 +218,6 @@ public class RestActionModule extends AbstractModule {
|
|||
bind(RestHealthAction.class).asEagerSingleton();
|
||||
bind(RestCatAction.class).asEagerSingleton();
|
||||
bind(RestHelpAction.class).asEagerSingleton();
|
||||
bind(org.elasticsearch.rest.action.cat.RestPendingClusterTasksAction.class).asEagerSingleton();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ public class RestHelpAction extends BaseRestHandler {
|
|||
s.append("/_cat/indices/{index}\n");
|
||||
s.append("/_cat/master\n");
|
||||
s.append("/_cat/nodes\n");
|
||||
s.append("/_cat/pending_tasks\n");
|
||||
s.append("/_cat/recovery\n");
|
||||
s.append("/_cat/shards\n");
|
||||
s.append("/_cat/shards/{index}\n");
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.elasticsearch.rest.action.cat;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksRequest;
|
||||
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.service.PendingClusterTask;
|
||||
import org.elasticsearch.common.Table;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.*;
|
||||
import org.elasticsearch.rest.action.support.RestTable;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
|
||||
public class RestPendingClusterTasksAction extends BaseRestHandler {
|
||||
@Inject
|
||||
public RestPendingClusterTasksAction(Settings settings, Client client, RestController controller) {
|
||||
super(settings, client);
|
||||
controller.registerHandler(GET, "/_cat/pending_tasks", this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleRequest(final RestRequest request, final RestChannel channel) {
|
||||
PendingClusterTasksRequest pendingClusterTasksRequest = new PendingClusterTasksRequest();
|
||||
client.admin().cluster().pendingClusterTasks(pendingClusterTasksRequest, new ActionListener<PendingClusterTasksResponse>() {
|
||||
@Override
|
||||
public void onResponse(PendingClusterTasksResponse pendingClusterTasks) {
|
||||
try {
|
||||
Table tab = buildTable(pendingClusterTasks);
|
||||
channel.sendResponse(RestTable.buildResponse(tab, request, channel));
|
||||
} catch (Throwable e) {
|
||||
onFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
try {
|
||||
channel.sendResponse(new XContentThrowableRestResponse(request, e));
|
||||
} catch (IOException e1) {
|
||||
logger.error("Failed to send failure response", e1);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Table buildTable(PendingClusterTasksResponse tasks) {
|
||||
Table t = new Table();
|
||||
|
||||
t.startHeaders();
|
||||
t.addCell("insertOrder", "text-align:right;desc:Task insertion order");
|
||||
t.addCell("timeInQueue", "text-align:right;desc:How long task has been in queue");
|
||||
t.addCell("priority", "desc:Task priority");
|
||||
t.addCell("source", "desc:Task source");
|
||||
t.endHeaders();
|
||||
|
||||
for (PendingClusterTask task : tasks) {
|
||||
t.startRow();
|
||||
t.addCell(task.getInsertOrder());
|
||||
t.addCell(task.getTimeInQueue());
|
||||
t.addCell(task.getPriority());
|
||||
t.addCell(task.getSource());
|
||||
t.endRow();
|
||||
}
|
||||
|
||||
return t;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue